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

[01/35] hive git commit: HIVE-15125: LLAP: Parallelize slider package generator (Gopal V, reviewed by Sergey Shelukhin)

Repository: hive
Updated Branches:
  refs/heads/hive-14535 a5ba17d5e -> 05879a8ea


HIVE-15125: LLAP: Parallelize slider package generator (Gopal V, 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/aa7c9cd6
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/aa7c9cd6
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/aa7c9cd6

Branch: refs/heads/hive-14535
Commit: aa7c9cd614804c0bf683745614f7a2b264ce72bf
Parents: a4a00b2
Author: Gopal V <go...@apache.org>
Authored: Tue Nov 15 14:52:26 2016 -0800
Committer: Gopal V <go...@apache.org>
Committed: Tue Nov 15 14:52:26 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hive/llap/cli/LlapServiceDriver.java | 685 +++++++++++--------
 1 file changed, 390 insertions(+), 295 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/aa7c9cd6/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
index 6f533df..dfd2f7b 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java
@@ -34,6 +34,13 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration;
@@ -71,6 +78,7 @@ import org.eclipse.jetty.server.ssl.SslSocketConnector;
 import org.json.JSONObject;
 
 import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 public class LlapServiceDriver {
 
@@ -153,11 +161,21 @@ public class LlapServiceDriver {
     }
   }
 
+  private static abstract class NamedCallable<T> implements Callable<T> {
+    public final String taskName;
+    public NamedCallable (String name) {
+      this.taskName = name;
+    }
+    public String getName() {
+      return taskName;
+    }
+  }
+
   private void run(String[] args) throws Exception {
     LlapOptionsProcessor optionsProcessor = new LlapOptionsProcessor();
-    LlapOptions options = optionsProcessor.processOptions(args);
+    final LlapOptions options = optionsProcessor.processOptions(args);
 
-    Properties propsDirectOptions = new Properties();
+    final Properties propsDirectOptions = new Properties();
 
     if (options == null) {
       // help
@@ -171,346 +189,418 @@ public class LlapServiceDriver {
       throw new Exception("Cannot load any configuration to run command");
     }
 
-    FileSystem fs = FileSystem.get(conf);
-    FileSystem lfs = FileSystem.getLocal(conf).getRawFileSystem();
-
-    // needed so that the file is actually loaded into configuration.
-    for (String f : NEEDED_CONFIGS) {
-      conf.addResource(f);
-      if (conf.getResource(f) == null) {
-        throw new Exception("Unable to find required config file: " + f);
-      }
-    }
-    for (String f : OPTIONAL_CONFIGS) {
-      conf.addResource(f);
-    }
-
-    conf.reloadConfiguration();
+    final long t0 = System.nanoTime();
 
-    populateConfWithLlapProperties(conf, options.getConfig());
+    final FileSystem fs = FileSystem.get(conf);
+    final FileSystem lfs = FileSystem.getLocal(conf).getRawFileSystem();
 
+    final ExecutorService executor =
+        Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors() / 2,
+            new ThreadFactoryBuilder().setNameFormat("llap-pkg-%d").build());
+    final CompletionService<Void> asyncRunner = new ExecutorCompletionService<Void>(executor);
 
-    if (options.getName() != null) {
-      // update service registry configs - caveat: this has nothing to do with the actual settings
-      // as read by the AM
-      // if needed, use --hiveconf llap.daemon.service.hosts=@llap0 to dynamically switch between
-      // instances
-      conf.set(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname, "@" + options.getName());
-      propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname,
-          "@" + options.getName());
-    }
-
-    if (options.getLogger() != null) {
-      HiveConf.setVar(conf, ConfVars.LLAP_DAEMON_LOGGER, options.getLogger());
-      propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_LOGGER.varname, options.getLogger());
-    }
+    try {
 
-    if (options.getSize() != -1) {
-      if (options.getCache() != -1) {
-        if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_MAPPED) == false) {
-          // direct heap allocations need to be safer
-          Preconditions.checkArgument(options.getCache() < options.getSize(),
-              "Cache size (" + humanReadableByteCount(options.getCache()) + ") has to be smaller" +
-                  " than the container sizing (" + humanReadableByteCount(options.getSize()) + ")");
-        } else if (options.getCache() < options.getSize()) {
-          LOG.warn("Note that this might need YARN physical memory monitoring to be turned off "
-              + "(yarn.nodemanager.pmem-check-enabled=false)");
+      // needed so that the file is actually loaded into configuration.
+      for (String f : NEEDED_CONFIGS) {
+        conf.addResource(f);
+        if (conf.getResource(f) == null) {
+          throw new Exception("Unable to find required config file: " + f);
         }
       }
-      if (options.getXmx() != -1) {
-        Preconditions.checkArgument(options.getXmx() < options.getSize(),
-            "Working memory (Xmx=" + humanReadableByteCount(options.getXmx()) + ") has to be" +
-                " smaller than the container sizing (" +
-                humanReadableByteCount(options.getSize()) + ")");
-      }
-      if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_DIRECT)
-          && false == HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_MAPPED)) {
-        // direct and not memory mapped
-        Preconditions.checkArgument(options.getXmx() + options.getCache() < options.getSize(),
-            "Working memory + cache (Xmx="+ humanReadableByteCount(options.getXmx()) +
-                " + cache=" + humanReadableByteCount(options.getCache()) + ")"
-                + " has to be smaller than the container sizing (" +
-                humanReadableByteCount(options.getSize()) + ")");
+      for (String f : OPTIONAL_CONFIGS) {
+        conf.addResource(f);
       }
-    }
 
-    // This parameter is read in package.py - and nowhere else. Does not need to be part of HiveConf - that's just confusing.
-    final long minAlloc = conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1);
-    long containerSize = -1;
-    if (options.getSize() != -1) {
-      containerSize = options.getSize() / (1024 * 1024);
-      Preconditions.checkArgument(containerSize >= minAlloc,
-          "Container size (" + humanReadableByteCount(options.getSize()) + ") should be greater" +
-              " than minimum allocation(" + humanReadableByteCount(minAlloc * 1024L * 1024L) + ")");
-      conf.setLong(ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB.varname, containerSize);
-      propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB.varname, String.valueOf(containerSize));
-    }
+      conf.reloadConfiguration();
 
-    if (options.getExecutors() != -1) {
-      conf.setLong(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname, options.getExecutors());
-      propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname, String.valueOf(options.getExecutors()));
-      // TODO: vcpu settings - possibly when DRFA works right
-    }
+      populateConfWithLlapProperties(conf, options.getConfig());
 
-    if (options.getIoThreads() != -1) {
-      conf.setLong(ConfVars.LLAP_IO_THREADPOOL_SIZE.varname, options.getIoThreads());
-      propsDirectOptions.setProperty(ConfVars.LLAP_IO_THREADPOOL_SIZE.varname,
-          String.valueOf(options.getIoThreads()));
-    }
+      if (options.getName() != null) {
+        // update service registry configs - caveat: this has nothing to do with the actual settings
+        // as read by the AM
+        // if needed, use --hiveconf llap.daemon.service.hosts=@llap0 to dynamically switch between
+        // instances
+        conf.set(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname, "@" + options.getName());
+        propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname,
+            "@" + options.getName());
+      }
 
-    long cache = -1, xmx = -1;
-    if (options.getCache() != -1) {
-      cache = options.getCache();
-      conf.set(HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname, Long.toString(cache));
-      propsDirectOptions.setProperty(HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname,
-          Long.toString(cache));
-    }
+      if (options.getLogger() != null) {
+        HiveConf.setVar(conf, ConfVars.LLAP_DAEMON_LOGGER, options.getLogger());
+        propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_LOGGER.varname, options.getLogger());
+      }
 
-    if (options.getXmx() != -1) {
-      // Needs more explanation here
-      // Xmx is not the max heap value in JDK8. You need to subtract 50% of the survivor fraction
-      // from this, to get actual usable  memory before it goes into GC
-      xmx = options.getXmx();
-      long xmxMb = (long)(xmx / (1024 * 1024));
-      conf.setLong(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname, xmxMb);
-      propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname,
-          String.valueOf(xmxMb));
-    }
+      if (options.getSize() != -1) {
+        if (options.getCache() != -1) {
+          if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_MAPPED) == false) {
+            // direct heap allocations need to be safer
+            Preconditions.checkArgument(options.getCache() < options.getSize(), "Cache size ("
+                + humanReadableByteCount(options.getCache()) + ") has to be smaller"
+                + " than the container sizing (" + humanReadableByteCount(options.getSize()) + ")");
+          } else if (options.getCache() < options.getSize()) {
+            LOG.warn("Note that this might need YARN physical memory monitoring to be turned off "
+                + "(yarn.nodemanager.pmem-check-enabled=false)");
+          }
+        }
+        if (options.getXmx() != -1) {
+          Preconditions.checkArgument(options.getXmx() < options.getSize(), "Working memory (Xmx="
+              + humanReadableByteCount(options.getXmx()) + ") has to be"
+              + " smaller than the container sizing (" + humanReadableByteCount(options.getSize())
+              + ")");
+        }
+        if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_DIRECT)
+            && false == HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_MAPPED)) {
+          // direct and not memory mapped
+          Preconditions.checkArgument(options.getXmx() + options.getCache() < options.getSize(),
+              "Working memory + cache (Xmx=" + humanReadableByteCount(options.getXmx())
+                  + " + cache=" + humanReadableByteCount(options.getCache()) + ")"
+                  + " has to be smaller than the container sizing ("
+                  + humanReadableByteCount(options.getSize()) + ")");
+        }
+      }
 
-    if (options.getLlapQueueName() != null && !options.getLlapQueueName().isEmpty()) {
-      conf.set(ConfVars.LLAP_DAEMON_QUEUE_NAME.varname, options.getLlapQueueName());
-      propsDirectOptions
-          .setProperty(ConfVars.LLAP_DAEMON_QUEUE_NAME.varname, options.getLlapQueueName());
-    }
+      // This parameter is read in package.py - and nowhere else. Does not need to be part of
+      // HiveConf - that's just confusing.
+      final long minAlloc = conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1);
+      long containerSize = -1;
+      if (options.getSize() != -1) {
+        containerSize = options.getSize() / (1024 * 1024);
+        Preconditions.checkArgument(containerSize >= minAlloc, "Container size ("
+            + humanReadableByteCount(options.getSize()) + ") should be greater"
+            + " than minimum allocation(" + humanReadableByteCount(minAlloc * 1024L * 1024L) + ")");
+        conf.setLong(ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB.varname, containerSize);
+        propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB.varname,
+            String.valueOf(containerSize));
+      }
 
-    URL logger = conf.getResource(LlapConstants.LOG4j2_PROPERTIES_FILE);
+      if (options.getExecutors() != -1) {
+        conf.setLong(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname, options.getExecutors());
+        propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname,
+            String.valueOf(options.getExecutors()));
+        // TODO: vcpu settings - possibly when DRFA works right
+      }
 
-    if (null == logger) {
-      throw new Exception("Unable to find required config file: llap-daemon-log4j2.properties");
-    }
+      if (options.getIoThreads() != -1) {
+        conf.setLong(ConfVars.LLAP_IO_THREADPOOL_SIZE.varname, options.getIoThreads());
+        propsDirectOptions.setProperty(ConfVars.LLAP_IO_THREADPOOL_SIZE.varname,
+            String.valueOf(options.getIoThreads()));
+      }
 
-    Path home = new Path(System.getenv("HIVE_HOME"));
-    Path scripts = new Path(new Path(new Path(home, "scripts"), "llap"), "bin");
+      long cache = -1, xmx = -1;
+      if (options.getCache() != -1) {
+        cache = options.getCache();
+        conf.set(HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname, Long.toString(cache));
+        propsDirectOptions.setProperty(HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname,
+            Long.toString(cache));
+      }
 
-    if (!lfs.exists(home)) {
-      throw new Exception("Unable to find HIVE_HOME:" + home);
-    } else if (!lfs.exists(scripts)) {
-      LOG.warn("Unable to find llap scripts:" + scripts);
-    }
+      if (options.getXmx() != -1) {
+        // Needs more explanation here
+        // Xmx is not the max heap value in JDK8. You need to subtract 50% of the survivor fraction
+        // from this, to get actual usable memory before it goes into GC
+        xmx = options.getXmx();
+        long xmxMb = (long) (xmx / (1024 * 1024));
+        conf.setLong(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname, xmxMb);
+        propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname,
+            String.valueOf(xmxMb));
+      }
 
+      if (options.getLlapQueueName() != null && !options.getLlapQueueName().isEmpty()) {
+        conf.set(ConfVars.LLAP_DAEMON_QUEUE_NAME.varname, options.getLlapQueueName());
+        propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_QUEUE_NAME.varname,
+            options.getLlapQueueName());
+      }
 
-    Path libDir = new Path(tmpDir, "lib");
-    Path tezDir = new Path(libDir, "tez");
-    Path udfDir = new Path(libDir, "udfs");
+      final URL logger = conf.getResource(LlapConstants.LOG4j2_PROPERTIES_FILE);
 
-    String tezLibs = conf.get(TezConfiguration.TEZ_LIB_URIS);
-    if (tezLibs == null) {
-      LOG.warn("Missing tez.lib.uris in tez-site.xml");
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Copying tez libs from " + tezLibs);
-    }
-    lfs.mkdirs(tezDir);
-    fs.copyToLocalFile(new Path(tezLibs), new Path(libDir, "tez.tar.gz"));
-    CompressionUtils.unTar(new Path(libDir, "tez.tar.gz").toString(), tezDir.toString(), true);
-    lfs.delete(new Path(libDir, "tez.tar.gz"), false);
-
-    Class<?>[] dependencies = new Class<?>[] {
-        LlapDaemonProtocolProtos.class, // llap-common
-        LlapTezUtils.class, // llap-tez
-        LlapInputFormat.class, // llap-server
-        HiveInputFormat.class, // hive-exec
-        SslSocketConnector.class, // hive-common (https deps)
-        RegistryUtils.ServiceRecordMarshal.class, // ZK registry
-        // log4j2
-        com.lmax.disruptor.RingBuffer.class, // disruptor
-        org.apache.logging.log4j.Logger.class, // log4j-api
-        org.apache.logging.log4j.core.Appender.class, // log4j-core
-        org.apache.logging.slf4j.Log4jLogger.class, // log4j-slf4j
-        // log4j-1.2-API needed for NDC
-        org.apache.log4j.NDC.class,
-    };
-
-    for (Class<?> c : dependencies) {
-      Path jarPath = new Path(Utilities.jarFinderGetJar(c));
-      lfs.copyFromLocalFile(jarPath, libDir);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Copying " + jarPath + " to " + libDir);
+      if (null == logger) {
+        throw new Exception("Unable to find required config file: llap-daemon-log4j2.properties");
       }
-    }
 
+      Path home = new Path(System.getenv("HIVE_HOME"));
+      Path scripts = new Path(new Path(new Path(home, "scripts"), "llap"), "bin");
 
-    // copy default aux classes (json/hbase)
-
-    for (String className : DEFAULT_AUX_CLASSES) {
-      localizeJarForClass(lfs, libDir, className, false);
-    }
-    Collection<String> codecs = conf.getStringCollection("io.compression.codecs");
-    if (codecs != null) {
-      for (String codecClassName : codecs) {
-        localizeJarForClass(lfs, libDir, codecClassName, false);
+      if (!lfs.exists(home)) {
+        throw new Exception("Unable to find HIVE_HOME:" + home);
+      } else if (!lfs.exists(scripts)) {
+        LOG.warn("Unable to find llap scripts:" + scripts);
       }
-    }
 
-    if (options.getIsHBase()) {
-      try {
-        localizeJarForClass(lfs, libDir, HBASE_SERDE_CLASS, true);
-        Job fakeJob = new Job(new JobConf()); // HBase API is convoluted.
-        TableMapReduceUtil.addDependencyJars(fakeJob);
-        Collection<String> hbaseJars = fakeJob.getConfiguration().getStringCollection("tmpjars");
-        for (String jarPath : hbaseJars) {
-          if (!jarPath.isEmpty()) {
-            lfs.copyFromLocalFile(new Path(jarPath), libDir);
+      final Path libDir = new Path(tmpDir, "lib");
+      final Path tezDir = new Path(libDir, "tez");
+      final Path udfDir = new Path(libDir, "udfs");
+      final Path confPath = new Path(tmpDir, "conf");
+      lfs.mkdirs(confPath);
+
+      NamedCallable<Void> downloadTez = new NamedCallable<Void>("downloadTez") {
+        @Override
+        public Void call() throws Exception {
+          synchronized (fs) {
+            String tezLibs = conf.get(TezConfiguration.TEZ_LIB_URIS);
+            if (tezLibs == null) {
+              LOG.warn("Missing tez.lib.uris in tez-site.xml");
+            }
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Copying tez libs from " + tezLibs);
+            }
+            lfs.mkdirs(tezDir);
+            fs.copyToLocalFile(new Path(tezLibs), new Path(libDir, "tez.tar.gz"));
+            CompressionUtils.unTar(new Path(libDir, "tez.tar.gz").toString(), tezDir.toString(),
+                true);
+            lfs.delete(new Path(libDir, "tez.tar.gz"), false);
           }
+          return null;
         }
-      } catch (Throwable t) {
-        String err = "Failed to add HBase jars. Use --auxhbase=false to avoid localizing them";
-        LOG.error(err);
-        System.err.println(err);
-        throw new RuntimeException(t);
-      }
-    }
-
-    String auxJars = options.getAuxJars();
-    if (auxJars != null && !auxJars.isEmpty()) {
-      // TODO: transitive dependencies warning?
-      String[] jarPaths = auxJars.split(",");
-      for (String jarPath : jarPaths) {
-        if (!jarPath.isEmpty()) {
-          lfs.copyFromLocalFile(new Path(jarPath), libDir);
+      };
+
+      NamedCallable<Void> copyLocalJars = new NamedCallable<Void>("copyLocalJars") {
+        @Override
+        public Void call() throws Exception {
+          Class<?>[] dependencies = new Class<?>[] { LlapDaemonProtocolProtos.class, // llap-common
+              LlapTezUtils.class, // llap-tez
+              LlapInputFormat.class, // llap-server
+              HiveInputFormat.class, // hive-exec
+              SslSocketConnector.class, // hive-common (https deps)
+              RegistryUtils.ServiceRecordMarshal.class, // ZK registry
+              // log4j2
+              com.lmax.disruptor.RingBuffer.class, // disruptor
+              org.apache.logging.log4j.Logger.class, // log4j-api
+              org.apache.logging.log4j.core.Appender.class, // log4j-core
+              org.apache.logging.slf4j.Log4jLogger.class, // log4j-slf4j
+              // log4j-1.2-API needed for NDC
+              org.apache.log4j.NDC.class, };
+
+          for (Class<?> c : dependencies) {
+            Path jarPath = new Path(Utilities.jarFinderGetJar(c));
+            lfs.copyFromLocalFile(jarPath, libDir);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Copying " + jarPath + " to " + libDir);
+            }
+          }
+          return null;
         }
-      }
-    }
+      };
 
-    // UDFs
-    final Set<String> allowedUdfs;
-    
-    if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOW_PERMANENT_FNS)) {
-      allowedUdfs = downloadPermanentFunctions(conf, udfDir);
-    } else {
-      allowedUdfs = Collections.emptySet();
-    }
+      // copy default aux classes (json/hbase)
 
-    String java_home;
-    if (options.getJavaPath() == null || options.getJavaPath().isEmpty()) {
-      java_home = System.getenv("JAVA_HOME");
-      String jre_home = System.getProperty("java.home");
-      if (java_home == null) {
-        java_home = jre_home;
-      } else if (!java_home.equals(jre_home)) {
-        LOG.warn("Java versions might not match : JAVA_HOME=[{}],process jre=[{}]",
-            java_home, jre_home);
-      }
-    } else {
-      java_home = options.getJavaPath();
-    }
-    if (java_home == null || java_home.isEmpty()) {
-      throw new RuntimeException(
-          "Could not determine JAVA_HOME from command line parameters, environment or system properties");
-    }
-    LOG.info("Using [{}] for JAVA_HOME", java_home);
+      NamedCallable<Void> copyAuxJars = new NamedCallable<Void>("copyAuxJars") {
+        @Override
+        public Void call() throws Exception {
+          for (String className : DEFAULT_AUX_CLASSES) {
+            localizeJarForClass(lfs, libDir, className, false);
+          }
+          Collection<String> codecs = conf.getStringCollection("io.compression.codecs");
+          if (codecs != null) {
+            for (String codecClassName : codecs) {
+              localizeJarForClass(lfs, libDir, codecClassName, false);
+            }
+          }
 
-    Path confPath = new Path(tmpDir, "conf");
-    lfs.mkdirs(confPath);
+          if (options.getIsHBase()) {
+            try {
+              localizeJarForClass(lfs, libDir, HBASE_SERDE_CLASS, true);
+              Job fakeJob = new Job(new JobConf()); // HBase API is convoluted.
+              TableMapReduceUtil.addDependencyJars(fakeJob);
+              Collection<String> hbaseJars =
+                  fakeJob.getConfiguration().getStringCollection("tmpjars");
+              for (String jarPath : hbaseJars) {
+                if (!jarPath.isEmpty()) {
+                  lfs.copyFromLocalFile(new Path(jarPath), libDir);
+                }
+              }
+            } catch (Throwable t) {
+              String err =
+                  "Failed to add HBase jars. Use --auxhbase=false to avoid localizing them";
+              LOG.error(err);
+              System.err.println(err);
+              throw new RuntimeException(t);
+            }
+          }
 
-    // Copy over the mandatory configs for the package.
-    for (String f : NEEDED_CONFIGS) {
-      copyConfig(lfs, confPath, f);
-    }
-    for (String f : OPTIONAL_CONFIGS) {
-      try {
-        copyConfig(lfs, confPath, f);
-      } catch (Throwable t) {
-        LOG.info("Error getting an optional config " + f + "; ignoring: " + t.getMessage());
-      }
-    }
-    createLlapDaemonConfig(lfs, confPath, conf, propsDirectOptions, options.getConfig());
-
-    // logger can be a resource stream or a real file (cannot use copy)
-    InputStream loggerContent = logger.openStream();
-    IOUtils.copyBytes(loggerContent,
-        lfs.create(new Path(confPath, "llap-daemon-log4j2.properties"), true), conf, true);
-
-    String metricsFile = LlapConstants.LLAP_HADOOP_METRICS2_PROPERTIES_FILE;
-    URL metrics2 = conf.getResource(metricsFile);
-    if (metrics2 == null) {
-      LOG.warn(LlapConstants.LLAP_HADOOP_METRICS2_PROPERTIES_FILE + " cannot be found." +
-          " Looking for " + LlapConstants.HADOOP_METRICS2_PROPERTIES_FILE);
-      metricsFile = LlapConstants.HADOOP_METRICS2_PROPERTIES_FILE;
-      metrics2 = conf.getResource(metricsFile);
-    }
-    if (metrics2 != null) {
-      InputStream metrics2FileStream = metrics2.openStream();
-      IOUtils.copyBytes(metrics2FileStream, lfs.create(new Path(confPath, metricsFile), true),
-          conf, true);
-      LOG.info("Copied hadoop metrics2 properties file from " + metrics2);
-    } else {
-      LOG.warn("Cannot find " + LlapConstants.LLAP_HADOOP_METRICS2_PROPERTIES_FILE + " or " +
-          LlapConstants.HADOOP_METRICS2_PROPERTIES_FILE + " in classpath.");
-    }
+          String auxJars = options.getAuxJars();
+          if (auxJars != null && !auxJars.isEmpty()) {
+            // TODO: transitive dependencies warning?
+            String[] jarPaths = auxJars.split(",");
+            for (String jarPath : jarPaths) {
+              if (!jarPath.isEmpty()) {
+                lfs.copyFromLocalFile(new Path(jarPath), libDir);
+              }
+            }
+          }
+          return null;
+        }
+      };
+
+      NamedCallable<Void> copyUdfJars = new NamedCallable<Void>("copyUdfJars") {
+        @Override
+        public Void call() throws Exception {
+          // UDFs
+          final Set<String> allowedUdfs;
+
+          if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOW_PERMANENT_FNS)) {
+            synchronized (fs) {
+              allowedUdfs = downloadPermanentFunctions(conf, udfDir);
+            }
+          } else {
+            allowedUdfs = Collections.emptySet();
+          }
 
-    PrintWriter udfStream =
-        new PrintWriter(lfs.create(new Path(confPath, StaticPermanentFunctionChecker.PERMANENT_FUNCTIONS_LIST)));
-    for (String udfClass : allowedUdfs) {
-      udfStream.println(udfClass);
-    }
-    
-    udfStream.close();
+          PrintWriter udfStream =
+              new PrintWriter(lfs.create(new Path(confPath,
+                  StaticPermanentFunctionChecker.PERMANENT_FUNCTIONS_LIST)));
+          for (String udfClass : allowedUdfs) {
+            udfStream.println(udfClass);
+          }
 
-    // extract configs for processing by the python fragments in Slider
-    JSONObject configs = new JSONObject();
+          udfStream.close();
+          return null;
+        }
+      };
+
+      String java_home;
+      if (options.getJavaPath() == null || options.getJavaPath().isEmpty()) {
+        java_home = System.getenv("JAVA_HOME");
+        String jre_home = System.getProperty("java.home");
+        if (java_home == null) {
+          java_home = jre_home;
+        } else if (!java_home.equals(jre_home)) {
+          LOG.warn("Java versions might not match : JAVA_HOME=[{}],process jre=[{}]", java_home,
+              jre_home);
+        }
+      } else {
+        java_home = options.getJavaPath();
+      }
+      if (java_home == null || java_home.isEmpty()) {
+        throw new RuntimeException(
+            "Could not determine JAVA_HOME from command line parameters, environment or system properties");
+      }
+      LOG.info("Using [{}] for JAVA_HOME", java_home);
+
+      NamedCallable<Void> copyConfigs = new NamedCallable<Void>("copyConfigs") {
+        @Override
+        public Void call() throws Exception {
+          // Copy over the mandatory configs for the package.
+          for (String f : NEEDED_CONFIGS) {
+            copyConfig(lfs, confPath, f);
+          }
+          for (String f : OPTIONAL_CONFIGS) {
+            try {
+              copyConfig(lfs, confPath, f);
+            } catch (Throwable t) {
+              LOG.info("Error getting an optional config " + f + "; ignoring: " + t.getMessage());
+            }
+          }
+          createLlapDaemonConfig(lfs, confPath, conf, propsDirectOptions, options.getConfig());
+
+          // logger can be a resource stream or a real file (cannot use copy)
+          InputStream loggerContent = logger.openStream();
+          IOUtils.copyBytes(loggerContent,
+              lfs.create(new Path(confPath, "llap-daemon-log4j2.properties"), true), conf, true);
+
+          String metricsFile = LlapConstants.LLAP_HADOOP_METRICS2_PROPERTIES_FILE;
+          URL metrics2 = conf.getResource(metricsFile);
+          if (metrics2 == null) {
+            LOG.warn(LlapConstants.LLAP_HADOOP_METRICS2_PROPERTIES_FILE + " cannot be found."
+                + " Looking for " + LlapConstants.HADOOP_METRICS2_PROPERTIES_FILE);
+            metricsFile = LlapConstants.HADOOP_METRICS2_PROPERTIES_FILE;
+            metrics2 = conf.getResource(metricsFile);
+          }
+          if (metrics2 != null) {
+            InputStream metrics2FileStream = metrics2.openStream();
+            IOUtils.copyBytes(metrics2FileStream,
+                lfs.create(new Path(confPath, metricsFile), true), conf, true);
+            LOG.info("Copied hadoop metrics2 properties file from " + metrics2);
+          } else {
+            LOG.warn("Cannot find " + LlapConstants.LLAP_HADOOP_METRICS2_PROPERTIES_FILE + " or "
+                + LlapConstants.HADOOP_METRICS2_PROPERTIES_FILE + " in classpath.");
+          }
+          return null;
+        }
+      };
+
+      @SuppressWarnings("unchecked")
+      final NamedCallable<Void>[] asyncWork =
+          new NamedCallable[] {
+          downloadTez,
+          copyUdfJars,
+          copyLocalJars,
+          copyAuxJars,
+          copyConfigs };
+      @SuppressWarnings("unchecked")
+      final Future<Void>[] asyncResults = new Future[asyncWork.length];
+      for (int i = 0; i < asyncWork.length; i++) {
+        asyncResults[i] = asyncRunner.submit(asyncWork[i]);
+      }
 
-    configs.put("java.home", java_home);
+      // extract configs for processing by the python fragments in Slider
+      JSONObject configs = new JSONObject();
 
-    configs.put(ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB.varname, HiveConf.getIntVar(conf,
-        ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB));
-    configs.put(ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB.varname, containerSize);
+      configs.put("java.home", java_home);
 
-    configs.put(HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname,
-        HiveConf.getSizeVar(conf, HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE));
+      configs.put(ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB.varname,
+          HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB));
+      configs.put(ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB.varname, containerSize);
 
-    configs.put(HiveConf.ConfVars.LLAP_ALLOCATOR_DIRECT.varname,
-        HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_DIRECT));
+      configs.put(HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname,
+          HiveConf.getSizeVar(conf, HiveConf.ConfVars.LLAP_IO_MEMORY_MAX_SIZE));
 
-    configs.put(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname, HiveConf.getIntVar(conf,
-        ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB));
+      configs.put(HiveConf.ConfVars.LLAP_ALLOCATOR_DIRECT.varname,
+          HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_DIRECT));
 
-    configs.put(ConfVars.LLAP_DAEMON_VCPUS_PER_INSTANCE.varname, HiveConf.getIntVar(conf,
-        ConfVars.LLAP_DAEMON_VCPUS_PER_INSTANCE));
+      configs.put(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname,
+          HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB));
 
-    configs.put(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname, HiveConf.getIntVar(conf,
-        ConfVars.LLAP_DAEMON_NUM_EXECUTORS));
+      configs.put(ConfVars.LLAP_DAEMON_VCPUS_PER_INSTANCE.varname,
+          HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_VCPUS_PER_INSTANCE));
 
-    // Let YARN pick the queue name, if it isn't provided in hive-site, or via the command-line
-    if (HiveConf.getVar(conf, ConfVars.LLAP_DAEMON_QUEUE_NAME) != null) {
-      configs.put(ConfVars.LLAP_DAEMON_QUEUE_NAME.varname,
-          HiveConf.getVar(conf, ConfVars.LLAP_DAEMON_QUEUE_NAME));
-    }
+      configs.put(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname,
+          HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_NUM_EXECUTORS));
 
-    // Propagate the cluster name to the script.
-    String clusterHosts = HiveConf.getVar(conf, ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
-    if (!StringUtils.isEmpty(clusterHosts) && clusterHosts.startsWith("@") &&
-        clusterHosts.length() > 1) {
-      configs.put(CONFIG_CLUSTER_NAME, clusterHosts.substring(1));
-    }
+      // Let YARN pick the queue name, if it isn't provided in hive-site, or via the command-line
+      if (HiveConf.getVar(conf, ConfVars.LLAP_DAEMON_QUEUE_NAME) != null) {
+        configs.put(ConfVars.LLAP_DAEMON_QUEUE_NAME.varname,
+            HiveConf.getVar(conf, ConfVars.LLAP_DAEMON_QUEUE_NAME));
+      }
 
-    configs.put(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-        conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1));
+      // Propagate the cluster name to the script.
+      String clusterHosts = HiveConf.getVar(conf, ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
+      if (!StringUtils.isEmpty(clusterHosts) && clusterHosts.startsWith("@")
+          && clusterHosts.length() > 1) {
+        configs.put(CONFIG_CLUSTER_NAME, clusterHosts.substring(1));
+      }
+
+      configs.put(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+          conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1));
 
-    configs.put(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
-        conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, -1));
+      configs.put(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+          conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, -1));
 
-    long maxDirect = (xmx > 0 && cache > 0 && xmx < cache * 1.25) ? (long)(cache * 1.25) : -1;
-    configs.put("max_direct_memory", Long.toString(maxDirect));
+      long maxDirect = (xmx > 0 && cache > 0 && xmx < cache * 1.25) ? (long) (cache * 1.25) : -1;
+      configs.put("max_direct_memory", Long.toString(maxDirect));
 
-    FSDataOutputStream os = lfs.create(new Path(tmpDir, "config.json"));
-    OutputStreamWriter w = new OutputStreamWriter(os);
-    configs.write(w);
-    w.close();
-    os.close();
+      FSDataOutputStream os = lfs.create(new Path(tmpDir, "config.json"));
+      OutputStreamWriter w = new OutputStreamWriter(os);
+      configs.write(w);
+      w.close();
+      os.close();
 
-    lfs.close();
-    fs.close();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Config generation took " + (System.nanoTime() - t0) + " ns");
+      }
+      for (int i = 0; i < asyncWork.length; i++) {
+        final long t1 = System.nanoTime();
+        asyncResults[i].get();
+        final long t2 = System.nanoTime();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(asyncWork[i].getName() + " waited for " + (t2 - t1) + " ns");
+        }
+      }
+    } finally {
+      executor.shutdown();
+      lfs.close();
+      fs.close();
+    }
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Exiting successfully");
@@ -520,7 +610,12 @@ public class LlapServiceDriver {
   private Set<String> downloadPermanentFunctions(Configuration conf, Path udfDir) throws HiveException,
       URISyntaxException, IOException {
     Map<String,String> udfs = new HashMap<String, String>();
-    Hive hive = Hive.get(false);
+    HiveConf hiveConf = new HiveConf();
+    // disable expensive operations on the metastore
+    hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_INIT_METADATA_COUNT_ENABLED, false);
+    hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_METRICS, false);
+    // performance problem: ObjectStore does its own new HiveConf()
+    Hive hive = Hive.getWithFastCheck(hiveConf, false);
     ResourceDownloader resourceDownloader =
         new ResourceDownloader(conf, udfDir.toUri().normalize().getPath());
     List<Function> fns = hive.getAllFunctions();


[03/35] hive git commit: HIVE-15208: Query string should be HTML encoded for Web UI (Jimmy Xiang, reviewed by Xuefu Zhang)

Posted by se...@apache.org.
HIVE-15208: Query string should be HTML encoded for Web UI (Jimmy Xiang, reviewed by Xuefu Zhang)


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

Branch: refs/heads/hive-14535
Commit: 5aff550b96f50bd65a37756ec19d66e0c8a8eceb
Parents: e4e3a07
Author: Jimmy Xiang <jx...@apache.org>
Authored: Tue Nov 15 09:41:24 2016 -0800
Committer: Jimmy Xiang <jx...@apache.org>
Committed: Wed Nov 16 09:08:51 2016 -0800

----------------------------------------------------------------------
 service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon  | 4 ++--
 service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp | 5 +++--
 2 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5aff550b/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
----------------------------------------------------------------------
diff --git a/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon b/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
index 690c6f3..07aa3c1 100644
--- a/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
+++ b/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
@@ -160,7 +160,7 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
         </%if>
         <tr>
             <td>Latency (s)</td>
-            <td><% sod.getRuntime()/1000 %></td>
+            <td><% sod.getRuntime() == null ? "Not finished" : sod.getRuntime()/1000 %></td>
         </tr>
     </table>
 </%def>
@@ -299,4 +299,4 @@ org.apache.hive.service.cli.operation.SQLOperationDisplay;
 <script src="/static/js/bootstrap.min.js" type="text/javascript"></script>
 <script src="/static/js/tab.js" type="text/javascript"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/hive/blob/5aff550b/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
----------------------------------------------------------------------
diff --git a/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp b/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
index 3c187b6..6e89981 100644
--- a/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
+++ b/service/src/resources/hive-webapps/hiveserver2/hiveserver2.jsp
@@ -31,6 +31,7 @@
   import="java.util.Collection"
   import="java.util.Date"
   import="java.util.List"
+  import="jodd.util.HtmlEncoder"
 %>
 
 <%
@@ -146,7 +147,7 @@ for (HiveSession hiveSession: hiveSessions) {
     %>
     <tr>
         <td><%= operation.getUserName() %></td>
-        <td><%= operation.getQueryDisplay() == null ? "Unknown" : operation.getQueryDisplay().getQueryString() %></td>
+        <td><%= HtmlEncoder.strict(operation.getQueryDisplay() == null ? "Unknown" : operation.getQueryDisplay().getQueryString()) %></td>
         <td><%= operation.getExecutionEngine() %>
         <td><%= operation.getState() %></td>
         <td><%= new Date(operation.getBeginTime()) %></td>
@@ -187,7 +188,7 @@ for (HiveSession hiveSession: hiveSessions) {
     %>
     <tr>
         <td><%= operation.getUserName() %></td>
-        <td><%= operation.getQueryDisplay() == null ? "Unknown" : operation.getQueryDisplay().getQueryString() %></td>
+        <td><%= HtmlEncoder.strict(operation.getQueryDisplay() == null ? "Unknown" : operation.getQueryDisplay().getQueryString()) %></td>
         <td><%= operation.getExecutionEngine() %>
         <td><%= operation.getState() %></td>
         <td><%= operation.getElapsedTime()/1000 %></td>


[07/35] hive git commit: HIVE-14982: Remove some reserved keywords in Hive 2.2 (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-14982: Remove some reserved keywords in Hive 2.2 (Pengcheng Xiong, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: 0025b9d25afb245c93c8d4d274daa0fba67d3ff8
Parents: c5d147b
Author: Pengcheng Xiong <px...@apache.org>
Authored: Thu Nov 17 10:46:32 2016 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Thu Nov 17 10:46:32 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  1 +
 .../clientpositive/nonReservedKeyWords.q        |  8 +++
 .../clientpositive/nonReservedKeyWords.q.out    | 51 ++++++++++++++++++++
 3 files changed, 60 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0025b9d2/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index a82083b..89dad86 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -763,6 +763,7 @@ nonReserved
     | KW_MERGE
     | KW_MATCHED
     | KW_REPL | KW_DUMP | KW_BATCH | KW_STATUS
+    | KW_CACHE | KW_DAYOFWEEK | KW_VIEWS
 ;
 
 //The following SQL2011 reserved keywords are used as function name only, but not as identifiers.

http://git-wip-us.apache.org/repos/asf/hive/blob/0025b9d2/ql/src/test/queries/clientpositive/nonReservedKeyWords.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/nonReservedKeyWords.q b/ql/src/test/queries/clientpositive/nonReservedKeyWords.q
new file mode 100644
index 0000000..c580c71
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/nonReservedKeyWords.q
@@ -0,0 +1,8 @@
+set hive.mapred.mode=nonstrict;
+
+explain CREATE TABLE CACHE (col STRING);
+
+explain CREATE TABLE DAYOFWEEK (col STRING);
+
+explain CREATE TABLE VIEWS (col STRING);
+

http://git-wip-us.apache.org/repos/asf/hive/blob/0025b9d2/ql/src/test/results/clientpositive/nonReservedKeyWords.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/nonReservedKeyWords.q.out b/ql/src/test/results/clientpositive/nonReservedKeyWords.q.out
new file mode 100644
index 0000000..4030c6d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/nonReservedKeyWords.q.out
@@ -0,0 +1,51 @@
+PREHOOK: query: explain CREATE TABLE CACHE (col STRING)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: explain CREATE TABLE CACHE (col STRING)
+POSTHOOK: type: CREATETABLE
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Create Table Operator:
+        Create Table
+          columns: col string
+          input format: org.apache.hadoop.mapred.TextInputFormat
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          name: default.CACHE
+
+PREHOOK: query: explain CREATE TABLE DAYOFWEEK (col STRING)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: explain CREATE TABLE DAYOFWEEK (col STRING)
+POSTHOOK: type: CREATETABLE
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Create Table Operator:
+        Create Table
+          columns: col string
+          input format: org.apache.hadoop.mapred.TextInputFormat
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          name: default.DAYOFWEEK
+
+PREHOOK: query: explain CREATE TABLE VIEWS (col STRING)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: explain CREATE TABLE VIEWS (col STRING)
+POSTHOOK: type: CREATETABLE
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+      Create Table Operator:
+        Create Table
+          columns: col string
+          input format: org.apache.hadoop.mapred.TextInputFormat
+          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+          serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          name: default.VIEWS
+


[09/35] hive git commit: HIVE-14815: Implement Parquet vectorization reader for Primitive types(Ferdinand Xu, review by Chao Sun) This closes #104

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/test/results/clientpositive/parquet_types_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_types_vectorization.q.out b/ql/src/test/results/clientpositive/parquet_types_vectorization.q.out
new file mode 100644
index 0000000..7818d73
--- /dev/null
+++ b/ql/src/test/results/clientpositive/parquet_types_vectorization.q.out
@@ -0,0 +1,850 @@
+PREHOOK: query: DROP TABLE parquet_types_staging
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE parquet_types_staging
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: DROP TABLE parquet_types
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE parquet_types
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE parquet_types_staging (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary string,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_types_staging
+POSTHOOK: query: CREATE TABLE parquet_types_staging (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary string,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_types_staging
+PREHOOK: query: CREATE TABLE parquet_types (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary binary,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_types
+POSTHOOK: query: CREATE TABLE parquet_types (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary binary,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_types
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/parquet_types.txt' OVERWRITE INTO TABLE parquet_types_staging
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@parquet_types_staging
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/parquet_types.txt' OVERWRITE INTO TABLE parquet_types_staging
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@parquet_types_staging
+PREHOOK: query: SELECT * FROM parquet_types_staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types_staging
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_types_staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types_staging
+#### A masked pattern was here ####
+100	1	1	1.0	0.0	abc	2011-01-01 01:01:01.111111111	a    	a  	B4F3CAFDBEDD	{"k1":"v1"}	[101,200]	{"c1":10,"c2":"a"}	2011-01-01
+101	2	2	1.1	0.3	def	2012-02-02 02:02:02.222222222	ab   	ab 	68692CCAC0BDE7	{"k2":"v2"}	[102,200]	{"c1":10,"c2":"d"}	2012-02-02
+102	3	3	1.2	0.6	ghi	2013-03-03 03:03:03.333333333	abc  	abc	B4F3CAFDBEDD	{"k3":"v3"}	[103,200]	{"c1":10,"c2":"g"}	2013-03-03
+103	1	4	1.3	0.9	jkl	2014-04-04 04:04:04.444444444	abcd 	abcd	68692CCAC0BDE7	{"k4":"v4"}	[104,200]	{"c1":10,"c2":"j"}	2014-04-04
+104	2	5	1.4	1.2	mno	2015-05-05 05:05:05.555555555	abcde	abcde	B4F3CAFDBEDD	{"k5":"v5"}	[105,200]	{"c1":10,"c2":"m"}	2015-05-05
+105	3	1	1.0	1.5	pqr	2016-06-06 06:06:06.666666666	abcde	abcdef	68692CCAC0BDE7	{"k6":"v6"}	[106,200]	{"c1":10,"c2":"p"}	2016-06-06
+106	1	2	1.1	1.8	stu	2017-07-07 07:07:07.777777777	abcde	abcdefg	B4F3CAFDBEDD	{"k7":"v7"}	[107,200]	{"c1":10,"c2":"s"}	2017-07-07
+107	2	3	1.2	2.1	vwx	2018-08-08 08:08:08.888888888	bcdef	abcdefgh	68692CCAC0BDE7	{"k8":"v8"}	[108,200]	{"c1":10,"c2":"v"}	2018-08-08
+108	3	4	1.3	2.4	yza	2019-09-09 09:09:09.999999999	cdefg	B4F3CAFDBE	68656C6C6F	{"k9":"v9"}	[109,200]	{"c1":10,"c2":"y"}	2019-09-09
+109	1	5	1.4	2.7	bcd	2020-10-10 10:10:10.101010101	klmno	abcdedef	68692CCAC0BDE7	{"k10":"v10"}	[110,200]	{"c1":10,"c2":"b"}	2020-10-10
+110	2	1	1.0	3.0	efg	2021-11-11 11:11:11.111111111	pqrst	abcdede	B4F3CAFDBEDD	{"k11":"v11"}	[111,200]	{"c1":10,"c2":"e"}	2021-11-11
+111	3	2	1.1	3.3	hij	2022-12-12 12:12:12.121212121	nopqr	abcded	68692CCAC0BDE7	{"k12":"v12"}	[112,200]	{"c1":10,"c2":"h"}	2022-12-12
+112	1	3	1.2	3.6	klm	2023-01-02 13:13:13.131313131	opqrs	abcdd	B4F3CAFDBEDD	{"k13":"v13"}	[113,200]	{"c1":10,"c2":"k"}	2023-01-02
+113	2	4	1.3	3.9	nop	2024-02-02 14:14:14.141414141	pqrst	abc	68692CCAC0BDE7	{"k14":"v14"}	[114,200]	{"c1":10,"c2":"n"}	2024-02-02
+114	3	5	1.4	4.2	qrs	2025-03-03 15:15:15.151515151	qrstu	b	B4F3CAFDBEDD	{"k15":"v15"}	[115,200]	{"c1":10,"c2":"q"}	2025-03-03
+115	1	1	1.0	4.5	qrs	2026-04-04 16:16:16.161616161	rstuv	abcded	68692CCAC0BDE7	{"k16":"v16"}	[116,200]	{"c1":10,"c2":"q"}	2026-04-04
+116	2	2	1.1	4.8	wxy	2027-05-05 17:17:17.171717171	stuvw	abcded	B4F3CAFDBEDD	{"k17":"v17"}	[117,200]	{"c1":10,"c2":"w"}	2027-05-05
+117	3	3	1.2	5.1	zab	2028-06-06 18:18:18.181818181	tuvwx	abcded	68692CCAC0BDE7	{"k18":"v18"}	[118,200]	{"c1":10,"c2":"z"}	2028-06-06
+118	1	4	1.3	5.4	cde	2029-07-07 19:19:19.191919191	uvwzy	abcdede	B4F3CAFDBEDD	{"k19":"v19"}	[119,200]	{"c1":10,"c2":"c"}	2029-07-07
+119	2	5	1.4	5.7	fgh	2030-08-08 20:20:20.202020202	vwxyz	abcdede	68692CCAC0BDE7	{"k20":"v20"}	[120,200]	{"c1":10,"c2":"f"}	2030-08-08
+120	3	1	1.0	6.0	ijk	2031-09-09 21:21:21.212121212	wxyza	abcde	B4F3CAFDBEDD	{"k21":"v21"}	[121,200]	{"c1":10,"c2":"i"}	2031-09-09
+121	1	2	1.1	6.3	lmn	2032-10-10 22:22:22.222222222	bcdef	abcde		{"k22":"v22"}	[122,200]	{"c1":10,"c2":"l"}	2032-10-10
+PREHOOK: query: INSERT OVERWRITE TABLE parquet_types
+SELECT cint, ctinyint, csmallint, cfloat, cdouble, cstring1, t, cchar, cvarchar,
+unhex(cbinary), m1, l1, st1, d FROM parquet_types_staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types_staging
+PREHOOK: Output: default@parquet_types
+POSTHOOK: query: INSERT OVERWRITE TABLE parquet_types
+SELECT cint, ctinyint, csmallint, cfloat, cdouble, cstring1, t, cchar, cvarchar,
+unhex(cbinary), m1, l1, st1, d FROM parquet_types_staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types_staging
+POSTHOOK: Output: default@parquet_types
+POSTHOOK: Lineage: parquet_types.cbinary EXPRESSION [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cbinary, type:string, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cchar SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cchar, type:char(5), comment:null), ]
+POSTHOOK: Lineage: parquet_types.cdouble SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cfloat SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cint SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_types.csmallint SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:csmallint, type:smallint, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cstring1 SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cstring1, type:string, comment:null), ]
+POSTHOOK: Lineage: parquet_types.ctinyint SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:ctinyint, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cvarchar SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cvarchar, type:varchar(10), comment:null), ]
+POSTHOOK: Lineage: parquet_types.d SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:d, type:date, comment:null), ]
+POSTHOOK: Lineage: parquet_types.l1 SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:l1, type:array<int>, comment:null), ]
+POSTHOOK: Lineage: parquet_types.m1 SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:m1, type:map<string,varchar(3)>, comment:null), ]
+POSTHOOK: Lineage: parquet_types.st1 SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:st1, type:struct<c1:int,c2:char(1)>, comment:null), ]
+POSTHOOK: Lineage: parquet_types.t SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:t, type:timestamp, comment:null), ]
+PREHOOK: query: -- test types in group by
+
+EXPLAIN SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- test types in group by
+
+EXPLAIN SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: ctinyint (type: tinyint), cint (type: int), csmallint (type: smallint), cstring1 (type: string), cfloat (type: float), cdouble (type: double)
+              outputColumnNames: ctinyint, cint, csmallint, cstring1, cfloat, cdouble
+              Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: max(cint), min(csmallint), count(cstring1), avg(cfloat), stddev_pop(cdouble)
+                keys: ctinyint (type: tinyint)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: tinyint)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: tinyint)
+                  Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,input:float>), _col5 (type: struct<count:bigint,sum:double,variance:double>)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: max(VALUE._col0), min(VALUE._col1), count(VALUE._col2), avg(VALUE._col3), stddev_pop(VALUE._col4)
+          keys: KEY._col0 (type: tinyint)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), round(_col4, 5) (type: double), round(_col5, 5) (type: double)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+            Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: tinyint)
+              sort order: +
+              Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: double), _col5 (type: double)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: tinyint), VALUE._col0 (type: int), VALUE._col1 (type: smallint), VALUE._col2 (type: bigint), VALUE._col3 (type: double), VALUE._col4 (type: double)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+1	121	1	8	1.175	2.06216
+2	119	1	7	1.21429	1.8
+3	120	1	7	1.17143	1.8
+PREHOOK: query: EXPLAIN SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cfloat (type: float)
+              outputColumnNames: cfloat
+              Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cfloat (type: float)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: float)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: float)
+                  Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: float)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: float)
+              sort order: +
+              Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: float), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+1.0	5
+1.1	5
+1.2	4
+1.3	4
+1.4	4
+PREHOOK: query: EXPLAIN SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cchar (type: char(5))
+              outputColumnNames: cchar
+              Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cchar (type: char(5))
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: char(5))
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: char(5))
+                  Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: char(5))
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: char(5))
+              sort order: +
+              Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: char(5)), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+a    	1
+ab   	1
+abc  	1
+abcd 	1
+abcde	3
+bcdef	2
+cdefg	1
+klmno	1
+nopqr	1
+opqrs	1
+pqrst	2
+qrstu	1
+rstuv	1
+stuvw	1
+tuvwx	1
+uvwzy	1
+vwxyz	1
+wxyza	1
+PREHOOK: query: EXPLAIN SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cvarchar (type: varchar(10))
+              outputColumnNames: cvarchar
+              Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cvarchar (type: varchar(10))
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: varchar(10))
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: varchar(10))
+                  Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: varchar(10))
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: varchar(10))
+              sort order: +
+              Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: varchar(10)), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+B4F3CAFDBE	1
+a  	1
+ab 	1
+abc	2
+abcd	1
+abcdd	1
+abcde	3
+abcded	4
+abcdede	3
+abcdedef	1
+abcdef	1
+abcdefg	1
+abcdefgh	1
+b	1
+PREHOOK: query: EXPLAIN SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cstring1 (type: string)
+              outputColumnNames: cstring1
+              Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cstring1 (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 22 Data size: 308 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: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: string)
+              sort order: +
+              Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+abc	1
+bcd	1
+cde	1
+def	1
+efg	1
+fgh	1
+ghi	1
+hij	1
+ijk	1
+jkl	1
+klm	1
+lmn	1
+mno	1
+nop	1
+pqr	1
+qrs	2
+stu	1
+vwx	1
+wxy	1
+yza	1
+zab	1
+PREHOOK: query: EXPLAIN SELECT t, count(*) FROM parquet_types GROUP BY t ORDER BY t
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT t, count(*) FROM parquet_types GROUP BY t ORDER BY t
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: t (type: timestamp)
+              outputColumnNames: t
+              Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: t (type: timestamp)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: timestamp)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: timestamp)
+                  Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: timestamp)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: timestamp)
+              sort order: +
+              Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: timestamp), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT t, count(*) FROM parquet_types GROUP BY t ORDER BY t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t, count(*) FROM parquet_types GROUP BY t ORDER BY t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+2011-01-01 01:01:01.111111111	1
+2012-02-02 02:02:02.222222222	1
+2013-03-03 03:03:03.333333333	1
+2014-04-04 04:04:04.444444444	1
+2015-05-05 05:05:05.555555555	1
+2016-06-06 06:06:06.666666666	1
+2017-07-07 07:07:07.777777777	1
+2018-08-08 08:08:08.888888888	1
+2019-09-09 09:09:09.999999999	1
+2020-10-10 10:10:10.101010101	1
+2021-11-11 11:11:11.111111111	1
+2022-12-12 12:12:12.121212121	1
+2023-01-02 13:13:13.131313131	1
+2024-02-02 14:14:14.141414141	1
+2025-03-03 15:15:15.151515151	1
+2026-04-04 16:16:16.161616161	1
+2027-05-05 17:17:17.171717171	1
+2028-06-06 18:18:18.181818181	1
+2029-07-07 19:19:19.191919191	1
+2030-08-08 20:20:20.202020202	1
+2031-09-09 21:21:21.212121212	1
+2032-10-10 22:22:22.222222222	1
+PREHOOK: query: EXPLAIN SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary
+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: parquet_types
+            Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cbinary (type: binary)
+              outputColumnNames: cbinary
+              Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cbinary (type: binary)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: binary)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: binary)
+                  Statistics: Num rows: 22 Data size: 308 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: binary)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: hex(_col0) (type: string), _col1 (type: bigint)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 11 Data size: 154 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+	1
+68656C6C6F	1
+68692CCAC0BDE7	10
+B4F3CAFDBEDD	10

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out b/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out
index 281fe93..f493102 100644
--- a/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_parquet_types.q.out
@@ -149,6 +149,7 @@ STAGE PLANS:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                     serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Execution mode: vectorized
 
   Stage: Stage-0
     Fetch Operator
@@ -216,6 +217,7 @@ STAGE PLANS:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                     serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Execution mode: vectorized
 
   Stage: Stage-0
     Fetch Operator
@@ -305,6 +307,7 @@ STAGE PLANS:
                   Map-reduce partition columns: _col0 (type: tinyint)
                   Statistics: Num rows: 22 Data size: 242 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,input:float>), _col5 (type: struct<count:bigint,sum:double,variance:double>), _col6 (type: decimal(4,2))
+      Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           aggregations: max(VALUE._col0), min(VALUE._col1), count(VALUE._col2), avg(VALUE._col3), stddev_pop(VALUE._col4), max(VALUE._col5)


[25/35] hive git commit: HIVE-14902: HiveServer2: Cleanup code which checks for ThriftJDBCSerde usage (Ziyang Zhao reviewed by Vaibhav Gumashta)

Posted by se...@apache.org.
HIVE-14902: HiveServer2: Cleanup code which checks for ThriftJDBCSerde usage (Ziyang Zhao reviewed by Vaibhav Gumashta)


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

Branch: refs/heads/hive-14535
Commit: bbcff5ea1c719438a4fb74cbe16c80a9787e278a
Parents: 3baca6c
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Fri Nov 18 14:54:27 2016 -0800
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Fri Nov 18 14:54:27 2016 -0800

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/Driver.java     |  5 +----
 .../apache/hadoop/hive/ql/exec/FileSinkOperator.java  |  5 +----
 .../apache/hadoop/hive/ql/parse/SemanticAnalyzer.java | 12 +++++++++++-
 .../org/apache/hadoop/hive/ql/parse/TaskCompiler.java | 11 ++++++++++-
 .../org/apache/hadoop/hive/ql/plan/FetchWork.java     | 14 ++++++++++++++
 .../org/apache/hadoop/hive/ql/plan/FileSinkDesc.java  | 14 ++++++++++++++
 .../hive/service/cli/operation/SQLOperation.java      |  5 +----
 7 files changed, 52 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bbcff5ea/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 b77948b..757c60c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -2184,10 +2184,7 @@ public class Driver implements CommandProcessor {
        * indeed written using ThriftJDBCBinarySerDe, read one row from the output sequence file,
        * since it is a blob of row batches.
        */
-      if (fetchTask.getWork().isHiveServerQuery() && HiveConf.getBoolVar(conf,
-          HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS)
-          && (fetchTask.getTblDesc().getSerdeClassName()
-              .equalsIgnoreCase(ThriftJDBCBinarySerDe.class.getName()))) {
+      if (fetchTask.getWork().isUsingThriftJDBCBinarySerDe()) {
         maxRows = 1;
       }
       fetchTask.setMaxRows(maxRows);

http://git-wip-us.apache.org/repos/asf/hive/blob/bbcff5ea/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index eeba6cd..54d619c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -72,7 +72,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.SubStructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
-import org.apache.hadoop.hive.serde2.thrift.ThriftJDBCBinarySerDe;
 import org.apache.hadoop.hive.shims.HadoopShims.StoragePolicyShim;
 import org.apache.hadoop.hive.shims.HadoopShims.StoragePolicyValue;
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -1023,9 +1022,7 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
       // If serializer is ThriftJDBCBinarySerDe, then it buffers rows to a certain limit (hive.server2.thrift.resultset.max.fetch.size)
       // and serializes the whole batch when the buffer is full. The serialize returns null if the buffer is not full
       // (the size of buffer is kept track of in the ThriftJDBCBinarySerDe).
-      if (conf.isHiveServerQuery() && HiveConf.getBoolVar(hconf,
-          HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS) &&
-          serializer.getClass().getName().equalsIgnoreCase(ThriftJDBCBinarySerDe.class.getName())) {
+      if (conf.isUsingThriftJDBCBinarySerDe()) {
           try {
             recordValue = serializer.serialize(null, inputObjInspectors[0]);
             if ( null != fpaths ) {

http://git-wip-us.apache.org/repos/asf/hive/blob/bbcff5ea/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 47feeaf..7d8b2bd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -6990,7 +6990,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       dpCtx,
       dest_path);
 
-    fileSinkDesc.setHiveServerQuery(SessionState.get().isHiveServerQuery());
+    boolean isHiveServerQuery = SessionState.get().isHiveServerQuery();
+    fileSinkDesc.setHiveServerQuery(isHiveServerQuery);
     // If this is an insert, update, or delete on an ACID table then mark that so the
     // FileSinkOperator knows how to properly write to it.
     if (destTableIsAcid) {
@@ -7033,6 +7034,15 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       fileSinkDesc.setStaticSpec(dpCtx.getSPPath());
     }
 
+    if (isHiveServerQuery &&
+      null != table_desc &&
+      table_desc.getSerdeClassName().equalsIgnoreCase(ThriftJDBCBinarySerDe.class.getName()) &&
+      HiveConf.getBoolVar(conf,HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS)) {
+        fileSinkDesc.setIsUsingThriftJDBCBinarySerDe(true);
+    } else {
+        fileSinkDesc.setIsUsingThriftJDBCBinarySerDe(false);
+    }
+
     Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
         fileSinkDesc, fsRS, input), inputRR);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/bbcff5ea/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index 97cf585..8ce8ea3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -181,9 +181,18 @@ public abstract class TaskCompiler {
       }
 
       FetchWork fetch = new FetchWork(loadFileDesc.getSourcePath(), resultTab, outerQueryLimit);
-      fetch.setHiveServerQuery(SessionState.get().isHiveServerQuery());
+      boolean isHiveServerQuery = SessionState.get().isHiveServerQuery();
+      fetch.setHiveServerQuery(isHiveServerQuery);
       fetch.setSource(pCtx.getFetchSource());
       fetch.setSink(pCtx.getFetchSink());
+      if (isHiveServerQuery &&
+        null != resultTab &&
+        resultTab.getSerdeClassName().equalsIgnoreCase(ThriftJDBCBinarySerDe.class.getName()) &&
+        HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS)) {
+          fetch.setIsUsingThriftJDBCBinarySerDe(true);
+      } else {
+          fetch.setIsUsingThriftJDBCBinarySerDe(false);
+      }
 
       pCtx.setFetchTask((FetchTask) TaskFactory.get(fetch, conf));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/bbcff5ea/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
index 8ea6440..2ecb6f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
@@ -65,6 +65,12 @@ public class FetchWork implements Serializable {
 
   private boolean isHiveServerQuery;
 
+  /**
+   * Whether is a HiveServer query, and the destination table is
+   * indeed written using ThriftJDBCBinarySerDe
+   */
+  private boolean isUsingThriftJDBCBinarySerDe = false;
+
   public boolean isHiveServerQuery() {
 	return isHiveServerQuery;
   }
@@ -73,6 +79,14 @@ public class FetchWork implements Serializable {
 	this.isHiveServerQuery = isHiveServerQuery;
   }
 
+  public boolean isUsingThriftJDBCBinarySerDe() {
+	  return isUsingThriftJDBCBinarySerDe;
+  }
+
+  public void setIsUsingThriftJDBCBinarySerDe(boolean isUsingThriftJDBCBinarySerDe) {
+	  this.isUsingThriftJDBCBinarySerDe = isUsingThriftJDBCBinarySerDe;
+  }
+
   public FetchWork() {
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/bbcff5ea/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
index 07ed4fd..4d9139b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
@@ -97,6 +97,12 @@ public class FileSinkDesc extends AbstractOperatorDesc {
   private Path destPath;
   private boolean isHiveServerQuery;
 
+  /**
+   * Whether is a HiveServer query, and the destination table is
+   * indeed written using ThriftJDBCBinarySerDe
+   */
+  private boolean isUsingThriftJDBCBinarySerDe = false;
+
   public FileSinkDesc() {
   }
 
@@ -166,6 +172,14 @@ public class FileSinkDesc extends AbstractOperatorDesc {
 	  this.isHiveServerQuery = isHiveServerQuery;
   }
 
+  public boolean isUsingThriftJDBCBinarySerDe() {
+	  return this.isUsingThriftJDBCBinarySerDe;
+  }
+
+  public void setIsUsingThriftJDBCBinarySerDe(boolean isUsingThriftJDBCBinarySerDe) {
+	  this.isUsingThriftJDBCBinarySerDe = isUsingThriftJDBCBinarySerDe;
+  }
+
   @Explain(displayName = "directory", explainLevels = { Level.EXTENDED })
   public Path getDirName() {
     return dirName;

http://git-wip-us.apache.org/repos/asf/hive/blob/bbcff5ea/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index ba02c9c..80a615d 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -466,10 +466,7 @@ public class SQLOperation extends ExecuteStatementOperation {
     FetchTask fetchTask = driver.getFetchTask();
     boolean isBlobBased = false;
 
-    if (fetchTask != null && fetchTask.getWork().isHiveServerQuery() && HiveConf.getBoolVar(queryState.getConf(),
-        HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS)
-        && (fetchTask.getTblDesc().getSerdeClassName().equalsIgnoreCase(ThriftJDBCBinarySerDe.class
-            .getName()))) {
+    if (fetchTask != null && fetchTask.getWork().isUsingThriftJDBCBinarySerDe()) {
       // Just fetch one blob if we've serialized thrift objects in final tasks
       maxRows = 1;
       isBlobBased = true;


[23/35] hive git commit: HIVE-13539: HiveHFileOutputFormat searching the wrong directory for HFiles (Tim Robertson, Chaoyu Tang, reviewed by Aihua Xu)

Posted by se...@apache.org.
HIVE-13539: HiveHFileOutputFormat searching the wrong directory for HFiles (Tim Robertson, Chaoyu Tang, reviewed by Aihua Xu)


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

Branch: refs/heads/hive-14535
Commit: cf87b0e244d141a461c6b423b198680c9e6250fc
Parents: 0a6fc63
Author: Chaoyu Tang <ct...@cloudera.com>
Authored: Fri Nov 18 15:56:49 2016 -0500
Committer: Chaoyu Tang <ct...@cloudera.com>
Committed: Fri Nov 18 15:56:49 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hive/hbase/HiveHFileOutputFormat.java       | 13 ++++++++++---
 hbase-handler/src/test/queries/positive/hbase_bulk.q   |  2 +-
 .../src/test/queries/positive/hbase_handler_bulk.q     |  3 +++
 3 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cf87b0e2/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java
index 0cf1acc..b6582f8 100644
--- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java
+++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HiveHFileOutputFormat.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.Progressable;
 
@@ -115,6 +116,7 @@ public class HiveHFileOutputFormat extends
           job.getConfiguration(), progressable);
 
     final Path outputdir = FileOutputFormat.getOutputPath(tac);
+    final Path taskAttemptOutputdir = FileOutputCommitter.getTaskAttemptPath(tac, outputdir);
     final org.apache.hadoop.mapreduce.RecordWriter<
       ImmutableBytesWritable, KeyValue> fileWriter = getFileWriter(tac);
 
@@ -148,7 +150,7 @@ public class HiveHFileOutputFormat extends
           // location specified by the user.
           FileSystem fs = outputdir.getFileSystem(jc);
           fs.mkdirs(columnFamilyPath);
-          Path srcDir = outputdir;
+          Path srcDir = taskAttemptOutputdir;
           for (;;) {
             FileStatus [] files = fs.listStatus(srcDir, FileUtils.STAGING_DIR_PATH_FILTER);
             if ((files == null) || (files.length == 0)) {
@@ -161,6 +163,11 @@ public class HiveHFileOutputFormat extends
             if (srcDir.getName().equals(columnFamilyName)) {
               break;
             }
+            if (files[0].isFile()) {
+              throw new IOException("No family directories found in " + taskAttemptOutputdir + ". "
+                  + "The last component in hfile path should match column family name "
+                  + columnFamilyName);
+            }
           }
           for (FileStatus regionFile : fs.listStatus(srcDir, FileUtils.STAGING_DIR_PATH_FILTER)) {
             fs.rename(
@@ -171,8 +178,8 @@ public class HiveHFileOutputFormat extends
           }
           // Hive actually wants a file as task output (not a directory), so
           // replace the empty directory with an empty file to keep it happy.
-          fs.delete(outputdir, true);
-          fs.createNewFile(outputdir);
+          fs.delete(taskAttemptOutputdir, true);
+          fs.createNewFile(taskAttemptOutputdir);
         } catch (InterruptedException ex) {
           throw new IOException(ex);
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/cf87b0e2/hbase-handler/src/test/queries/positive/hbase_bulk.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_bulk.q b/hbase-handler/src/test/queries/positive/hbase_bulk.q
index f8bb47d..475aafc 100644
--- a/hbase-handler/src/test/queries/positive/hbase_bulk.q
+++ b/hbase-handler/src/test/queries/positive/hbase_bulk.q
@@ -32,7 +32,7 @@ from src
 where value='val_100' or value='val_200';
 
 dfs -count /tmp/data/hbpartition;
-dfs -cp /tmp/data/hbpartition/* /tmp/hbpartition.lst;
+dfs -cp -f /tmp/data/hbpartition/* /tmp/hbpartition.lst;
 
 set mapred.reduce.tasks=3;
 set hive.mapred.partitioner=org.apache.hadoop.mapred.lib.TotalOrderPartitioner;

http://git-wip-us.apache.org/repos/asf/hive/blob/cf87b0e2/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q b/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q
index f03da63..85581ec 100644
--- a/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q
+++ b/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q
@@ -10,6 +10,9 @@ tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk');
 
 set hive.hbase.generatehfiles=true;
 set hfile.family.path=/tmp/hb_target/cf;
+set mapreduce.input.fileinputformat.split.maxsize=200;
+set mapreduce.input.fileinputformat.split.minsize=200;
+set mapred.reduce.tasks=2;
 
 -- this should produce three files in /tmp/hb_target/cf
 insert overwrite table hb_target select distinct key, value from src cluster by key;


[02/35] hive git commit: HIVE-15129: LLAP : Enhance cache hits for stripe metadata across queries (Rajesh Balamohan reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-15129: LLAP : Enhance cache hits for stripe metadata across queries (Rajesh Balamohan 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/e4e3a07d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e4e3a07d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e4e3a07d

Branch: refs/heads/hive-14535
Commit: e4e3a07d8e46b70e35054fe15db3f2526952b877
Parents: aa7c9cd
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Nov 15 22:43:21 2016 -0800
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Nov 15 22:43:21 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hive/llap/io/metadata/OrcStripeMetadata.java  | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e4e3a07d/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java
index bc87094..6f0b9ff 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcStripeMetadata.java
@@ -102,8 +102,17 @@ public class OrcStripeMetadata extends LlapCacheableBuffer {
 
   public void loadMissingIndexes(DataReader mr, StripeInformation stripe, boolean[] includes,
       boolean[] sargColumns) throws IOException {
+    // Do not loose the old indexes. Create a super set includes
+    OrcProto.RowIndex[] existing = getRowIndexes();
+    boolean superset[] = new boolean[Math.max(existing.length, includes.length)];
+    for (int i = 0; i < includes.length; i++) {
+      superset[i] = includes[i];
+    }
+    for (int i = 0; i < existing.length; i++) {
+      superset[i] = superset[i] || (existing[i] != null);
+    }
     // TODO: should we save footer to avoid a read here?
-    rowIndex = mr.readRowIndex(stripe, null, includes, rowIndex.getRowGroupIndex(),
+    rowIndex = mr.readRowIndex(stripe, null, superset, rowIndex.getRowGroupIndex(),
         sargColumns, rowIndex.getBloomFilterIndex());
     // TODO: theoretically, we should re-estimate memory usage here and update memory manager
   }


[31/35] hive git commit: HIVE-15211: Provide support for complex expressions in ON clauses for INNER joins (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/893b2553/ql/src/test/results/clientpositive/join45.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join45.q.out b/ql/src/test/results/clientpositive/join45.q.out
new file mode 100644
index 0000000..18a7876
--- /dev/null
+++ b/ql/src/test/results/clientpositive/join45.q.out
@@ -0,0 +1,1771 @@
+PREHOOK: query: -- Conjunction with pred on multiple inputs and single inputs
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  AND src1.value between 100 and 102
+  AND src.value between 100 and 102)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Conjunction with pred on multiple inputs and single inputs
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  AND src1.value between 100 and 102
+  AND src.value between 100 and 102)
+LIMIT 10
+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: src1
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: _col1 BETWEEN 100 AND 102 (type: boolean)
+                  Statistics: Num rows: 2 Data size: 15 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: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: string)
+          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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: _col1 BETWEEN 100 AND 102 (type: boolean)
+                  Statistics: Num rows: 55 Data size: 584 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: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 60 Data size: 642 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 10
+            Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  AND src1.value between 100 and 102
+  AND src.value between 100 and 102)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  AND src1.value between 100 and 102
+  AND src.value between 100 and 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+PREHOOK: query: -- Conjunction with pred on multiple inputs and none
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key AND true)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Conjunction with pred on multiple inputs and none
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key AND true)
+LIMIT 10
+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: src1
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 25 Data size: 191 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: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: string)
+          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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 10
+            Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key AND true)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key AND true)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+128		128	val_128
+128		128	val_128
+128		128	val_128
+146	val_146	146	val_146
+146	val_146	146	val_146
+150	val_150	150	val_150
+213	val_213	213	val_213
+213	val_213	213	val_213
+224		224	val_224
+224		224	val_224
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: -- Conjunction with pred on single inputs and none
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.value between 100 and 102
+  AND src.value between 100 and 102
+  AND true)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Conjunction with pred on single inputs and none
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.value between 100 and 102
+  AND src.value between 100 and 102
+  AND true)
+LIMIT 10
+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: src1
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Filter Operator
+                predicate: _col1 BETWEEN 100 AND 102 (type: boolean)
+                Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Filter Operator
+                predicate: _col1 BETWEEN 100 AND 102 (type: boolean)
+                Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 110 Data size: 2103 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 10
+            Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.value between 100 and 102
+  AND src.value between 100 and 102
+  AND true)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.value between 100 and 102
+  AND src.value between 100 and 102
+  AND true)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: -- Disjunction with pred on multiple inputs and single inputs
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  OR src1.value between 100 and 102
+  OR src.value between 100 and 102)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Disjunction with pred on multiple inputs and single inputs
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  OR src1.value between 100 and 102
+  OR src.value between 100 and 102)
+LIMIT 10
+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: src1
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 12500 Data size: 240800 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: ((_col0 = _col2) or _col1 BETWEEN 100 AND 102 or _col3 BETWEEN 100 AND 102) (type: boolean)
+            Statistics: Num rows: 9026 Data size: 173876 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 10
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  OR src1.value between 100 and 102
+  OR src.value between 100 and 102)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  OR src1.value between 100 and 102
+  OR src.value between 100 and 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+406	val_406	406	val_406
+406	val_406	406	val_406
+406	val_406	406	val_406
+406	val_406	406	val_406
+146	val_146	146	val_146
+146	val_146	146	val_146
+213	val_213	213	val_213
+213	val_213	213	val_213
+128		128	val_128
+128		128	val_128
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: -- Conjunction with multiple inputs on one side
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  AND src1.key+src.key <= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Conjunction with multiple inputs on one side
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  AND src1.key+src.key <= 102)
+LIMIT 10
+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: src1
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 12500 Data size: 240800 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: (((UDFToDouble(_col0) + UDFToDouble(_col2)) <= 102.0) and ((UDFToDouble(_col0) + UDFToDouble(_col2)) >= 100.0)) (type: boolean)
+            Statistics: Num rows: 1388 Data size: 26738 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 10
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  AND src1.key+src.key <= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  AND src1.key+src.key <= 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+66	val_66	35	val_35
+66	val_66	34	val_34
+66	val_66	35	val_35
+66	val_66	35	val_35
+98	val_98	2	val_2
+98	val_98	4	val_4
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: -- Disjunction with multiple inputs on one side
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  OR src1.key+src.key <= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Disjunction with multiple inputs on one side
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  OR src1.key+src.key <= 102)
+LIMIT 10
+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: src1
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 12500 Data size: 240800 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: (((UDFToDouble(_col0) + UDFToDouble(_col2)) >= 100.0) or ((UDFToDouble(_col0) + UDFToDouble(_col2)) <= 102.0)) (type: boolean)
+            Statistics: Num rows: 8332 Data size: 160507 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 10
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  OR src1.key+src.key <= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  OR src1.key+src.key <= 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+406	val_406	97	val_97
+406	val_406	200	val_200
+406	val_406	400	val_400
+406	val_406	403	val_403
+406	val_406	169	val_169
+406	val_406	90	val_90
+406	val_406	126	val_126
+406	val_406	222	val_222
+406	val_406	477	val_477
+406	val_406	414	val_414
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: -- Function with multiple inputs on one side
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON ((src1.key,src.key) IN ((100,100),(101,101),(102,102)))
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Function with multiple inputs on one side
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON ((src1.key,src.key) IN ((100,100),(101,101),(102,102)))
+LIMIT 10
+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: src1
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 12500 Data size: 240800 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: (struct(_col0,_col2)) IN (const struct(100,100), const struct(101,101), const struct(102,102)) (type: boolean)
+            Statistics: Num rows: 3125 Data size: 60200 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 10
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src1 JOIN src
+ON ((src1.key,src.key) IN ((100,100),(101,101),(102,102)))
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src1 JOIN src
+ON ((src1.key,src.key) IN ((100,100),(101,101),(102,102)))
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: -- Chained 1
+EXPLAIN
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+LEFT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Chained 1
+EXPLAIN
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+LEFT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      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), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+          TableScan
+            alias: b
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 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: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+          TableScan
+            alias: a
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 13750 Data size: 264875 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: ((UDFToDouble(_col4) + UDFToDouble(_col0)) >= 100.0) (type: boolean)
+            Statistics: Num rows: 4583 Data size: 88285 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string), _col2 (type: string), _col3 (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+              Statistics: Num rows: 4583 Data size: 88285 Basic stats: COMPLETE Column stats: NONE
+              Limit
+                Number of rows: 10
+                Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+LEFT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+LEFT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+98	val_98	406	val_406	98	val_98
+98	val_98	146	val_146	98	val_98
+98	val_98	213	val_213	98	val_98
+98	val_98	128		98	val_98
+98	val_98	66	val_66	98	val_98
+98	val_98	369		98	val_98
+98	val_98	224		98	val_98
+98	val_98	273	val_273	98	val_98
+98	val_98	150	val_150	98	val_98
+98	val_98	401	val_401	98	val_98
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: -- Chained 2
+EXPLAIN
+SELECT *
+FROM src
+LEFT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Chained 2
+EXPLAIN
+SELECT *
+FROM src
+LEFT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      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), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+          TableScan
+            alias: a
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 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: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+          TableScan
+            alias: b
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 13750 Data size: 264875 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: ((UDFToDouble(_col4) + UDFToDouble(_col0)) <= 102.0) (type: boolean)
+            Statistics: Num rows: 4583 Data size: 88285 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 10
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src
+LEFT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src
+LEFT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+9	val_9	NULL	NULL	66	val_66
+8	val_8	NULL	NULL	66	val_66
+5	val_5	NULL	NULL	66	val_66
+5	val_5	NULL	NULL	66	val_66
+5	val_5	NULL	NULL	66	val_66
+4	val_4	NULL	NULL	66	val_66
+4	val_4	NULL	NULL	98	val_98
+35	val_35	NULL	NULL	66	val_66
+35	val_35	NULL	NULL	66	val_66
+35	val_35	NULL	NULL	66	val_66
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: -- Chained 3
+EXPLAIN
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+RIGHT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Chained 3
+EXPLAIN
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+RIGHT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      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), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: a
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 12500 Data size: 240800 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: ((UDFToDouble(_col2) + UDFToDouble(_col0)) >= 100.0) (type: boolean)
+            Statistics: Num rows: 4166 Data size: 80253 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: string)
+              Statistics: Num rows: 4166 Data size: 80253 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
+          TableScan
+            alias: b
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 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: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Right Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 4582 Data size: 88278 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 10
+            Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+RIGHT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+RIGHT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+NULL	NULL	NULL	NULL		val_484
+NULL	NULL	NULL	NULL		
+NULL	NULL	NULL	NULL		
+NULL	NULL	NULL	NULL		
+NULL	NULL	NULL	NULL		val_27
+NULL	NULL	NULL	NULL		val_165
+NULL	NULL	NULL	NULL		val_409
+NULL	NULL	NULL	NULL		
+NULL	NULL	NULL	NULL		val_193
+NULL	NULL	NULL	NULL		val_265
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: -- Chained 4
+EXPLAIN
+SELECT *
+FROM src
+RIGHT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Chained 4
+EXPLAIN
+SELECT *
+FROM src
+RIGHT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      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), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+          TableScan
+            alias: a
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 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: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Right Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+          TableScan
+            alias: b
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 13750 Data size: 264875 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: ((UDFToDouble(_col4) + UDFToDouble(_col0)) <= 102.0) (type: boolean)
+            Statistics: Num rows: 4583 Data size: 88285 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 10
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src
+RIGHT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src
+RIGHT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: -- Chained 5
+EXPLAIN
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+FULL OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Chained 5
+EXPLAIN
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+FULL OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      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), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+          TableScan
+            alias: a
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 12500 Data size: 240800 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: ((UDFToDouble(_col2) + UDFToDouble(_col0)) >= 100.0) (type: boolean)
+            Statistics: Num rows: 4166 Data size: 80253 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: string)
+              Statistics: Num rows: 4166 Data size: 80253 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
+          TableScan
+            alias: b
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 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: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Outer Join 0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 4582 Data size: 88278 Basic stats: COMPLETE Column stats: NONE
+          Limit
+            Number of rows: 10
+            Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+FULL OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+FULL OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+NULL	NULL	NULL	NULL		val_484
+NULL	NULL	NULL	NULL		
+NULL	NULL	NULL	NULL		
+NULL	NULL	NULL	NULL		
+NULL	NULL	NULL	NULL		val_27
+NULL	NULL	NULL	NULL		val_165
+NULL	NULL	NULL	NULL		val_409
+NULL	NULL	NULL	NULL		
+NULL	NULL	NULL	NULL		val_193
+NULL	NULL	NULL	NULL		val_265
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: -- Chained 6
+EXPLAIN
+SELECT *
+FROM src
+FULL OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Chained 6
+EXPLAIN
+SELECT *
+FROM src
+FULL OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      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), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+          TableScan
+            alias: a
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 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: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Outer Join 0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+          TableScan
+            alias: b
+            Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 13750 Data size: 264875 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: ((UDFToDouble(_col4) + UDFToDouble(_col0)) <= 102.0) (type: boolean)
+            Statistics: Num rows: 4583 Data size: 88285 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 10
+              Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 10 Data size: 190 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM src
+FULL OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM src
+FULL OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+9	val_9	NULL	NULL	66	val_66
+8	val_8	NULL	NULL	66	val_66
+5	val_5	NULL	NULL	66	val_66
+5	val_5	NULL	NULL	66	val_66
+5	val_5	NULL	NULL	66	val_66
+4	val_4	NULL	NULL	66	val_66
+4	val_4	NULL	NULL	98	val_98
+35	val_35	NULL	NULL	66	val_66
+35	val_35	NULL	NULL	66	val_66
+35	val_35	NULL	NULL	66	val_66
+Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: -- Right outer join with multiple inner joins and mixed conditions
+EXPLAIN
+SELECT *
+FROM cbo_t1 t1
+RIGHT OUTER JOIN cbo_t1 t2 ON (t2.key = t1.key)
+JOIN cbo_t1 t3 ON (t3.key = t2.key or t3.value = t2.value and t2.c_int = t1.c_int)
+JOIN cbo_t1 t4 ON (t4.key = t2.key or  t2.c_float = t4.c_float and t4.value = t2.value)
+JOIN cbo_t1 t5 ON (t5.key = t2.key or  t2.c_boolean = t4.c_boolean and t5.c_int = 42)
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Right outer join with multiple inner joins and mixed conditions
+EXPLAIN
+SELECT *
+FROM cbo_t1 t1
+RIGHT OUTER JOIN cbo_t1 t2 ON (t2.key = t1.key)
+JOIN cbo_t1 t3 ON (t3.key = t2.key or t3.value = t2.value and t2.c_int = t1.c_int)
+JOIN cbo_t1 t4 ON (t4.key = t2.key or  t2.c_float = t4.c_float and t4.value = t2.value)
+JOIN cbo_t1 t5 ON (t5.key = t2.key or  t2.c_boolean = t4.c_boolean and t5.c_int = 42)
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-4 is a root stage
+  Stage-1 depends on stages: Stage-4
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string), c_int (type: int), c_float (type: float), c_boolean (type: boolean), dt (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+              Statistics: Num rows: 20 Data size: 262 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: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string)
+          TableScan
+            alias: t2
+            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string), c_int (type: int), c_float (type: float), c_boolean (type: boolean), dt (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+              Statistics: Num rows: 20 Data size: 262 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: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Right Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+          Statistics: Num rows: 22 Data size: 288 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t3
+            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string), c_int (type: int), c_float (type: float), c_boolean (type: boolean), dt (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+              Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string)
+          TableScan
+            alias: t5
+            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string), c_int (type: int), c_float (type: float), c_boolean (type: boolean), dt (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+              Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string)
+          TableScan
+            alias: t4
+            Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string), c_int (type: int), c_float (type: float), c_boolean (type: boolean), dt (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+              Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                sort order: 
+                Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string)
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 22 Data size: 288 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: int), _col9 (type: float), _col10 (type: boolean), _col11 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Inner Join 0 to 1
+               Inner Join 0 to 2
+               Inner Join 0 to 3
+          keys:
+            0 
+            1 
+            2 
+            3 
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29
+          Statistics: Num rows: 176000 Data size: 9396800 Basic stats: COMPLETE Column stats: NONE
+          Filter Operator
+            predicate: (((_col0 = _col24) or ((_col1 = _col25) and (_col26 = _col20))) and ((_col12 = _col24) or ((_col27 = _col15) and (_col13 = _col25))) and ((_col6 = _col24) or ((_col28 = _col16) and (_col8 = 42)))) (type: boolean)
+            Statistics: Num rows: 74250 Data size: 3964275 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: _col18 (type: string), _col19 (type: string), _col20 (type: int), _col21 (type: float), _col22 (type: boolean), _col23 (type: string), _col24 (type: string), _col25 (type: string), _col26 (type: int), _col27 (type: float), _col28 (type: boolean), _col29 (type: string), _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: float), _col16 (type: boolean), _col17 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: int), _col9 (type: float), _col10 (type: boolean), _col11 (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29
+              Statistics: Num rows: 74250 Data size: 3964275 Basic stats: COMPLETE Column stats: NONE
+              Limit
+                Number of rows: 10
+                Statistics: Num rows: 10 Data size: 530 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 10 Data size: 530 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 10
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: SELECT *
+FROM cbo_t1 t1
+RIGHT OUTER JOIN cbo_t1 t2 ON (t2.key = t1.key)
+JOIN cbo_t1 t3 ON (t3.key = t2.key or t3.value = t2.value and t2.c_int = t1.c_int)
+JOIN cbo_t1 t4 ON (t4.key = t2.key or  t2.c_float = t4.c_float and t4.value = t2.value)
+JOIN cbo_t1 t5 ON (t5.key = t2.key or  t2.c_boolean = t4.c_boolean and t5.c_int = 42)
+LIMIT 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cbo_t1
+PREHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM cbo_t1 t1
+RIGHT OUTER JOIN cbo_t1 t2 ON (t2.key = t1.key)
+JOIN cbo_t1 t3 ON (t3.key = t2.key or t3.value = t2.value and t2.c_int = t1.c_int)
+JOIN cbo_t1 t4 ON (t4.key = t2.key or  t2.c_float = t4.c_float and t4.value = t2.value)
+JOIN cbo_t1 t5 ON (t5.key = t2.key or  t2.c_boolean = t4.c_boolean and t5.c_int = 42)
+LIMIT 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cbo_t1
+POSTHOOK: Input: default@cbo_t1@dt=2014
+#### A masked pattern was here ####
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014
+ 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014	 1 	 1 	1	1.0	true	2014


[17/35] hive git commit: HIVE-14089 : complex type support in LLAP IO is broken (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/test/results/clientpositive/vector_complex_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_complex_all.q.out b/ql/src/test/results/clientpositive/vector_complex_all.q.out
index a54a371..7ce707a 100644
--- a/ql/src/test/results/clientpositive/vector_complex_all.q.out
+++ b/ql/src/test/results/clientpositive/vector_complex_all.q.out
@@ -34,8 +34,9 @@ PREHOOK: query: CREATE TABLE orc_create_complex (
   str STRING,
   mp  MAP<STRING,STRING>,
   lst ARRAY<STRING>,
-  strct STRUCT<A:STRING,B:STRING>
-) STORED AS ORC
+  strct STRUCT<A:STRING,B:STRING>,
+  val string
+) STORED AS ORC tblproperties("orc.row.index.stride"="1000", "orc.stripe.size"="1000", "orc.compress.size"="10000")
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@orc_create_complex
@@ -43,16 +44,19 @@ POSTHOOK: query: CREATE TABLE orc_create_complex (
   str STRING,
   mp  MAP<STRING,STRING>,
   lst ARRAY<STRING>,
-  strct STRUCT<A:STRING,B:STRING>
-) STORED AS ORC
+  strct STRUCT<A:STRING,B:STRING>,
+  val string
+) STORED AS ORC tblproperties("orc.row.index.stride"="1000", "orc.stripe.size"="1000", "orc.compress.size"="10000")
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@orc_create_complex
-PREHOOK: query: INSERT OVERWRITE TABLE orc_create_complex SELECT * FROM orc_create_staging
+PREHOOK: query: INSERT OVERWRITE TABLE orc_create_complex
+SELECT orc_create_staging.*, '0' FROM orc_create_staging
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_staging
 PREHOOK: Output: default@orc_create_complex
-POSTHOOK: query: INSERT OVERWRITE TABLE orc_create_complex SELECT * FROM orc_create_staging
+POSTHOOK: query: INSERT OVERWRITE TABLE orc_create_complex
+SELECT orc_create_staging.*, '0' FROM orc_create_staging
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_staging
 POSTHOOK: Output: default@orc_create_complex
@@ -60,45 +64,8 @@ POSTHOOK: Lineage: orc_create_complex.lst SIMPLE [(orc_create_staging)orc_create
 POSTHOOK: Lineage: orc_create_complex.mp SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:mp, type:map<string,string>, comment:null), ]
 POSTHOOK: Lineage: orc_create_complex.str SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:str, type:string, comment:null), ]
 POSTHOOK: Lineage: orc_create_complex.strct SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:strct, type:struct<A:string,B:string>, comment:null), ]
-orc_create_staging.str	orc_create_staging.mp	orc_create_staging.lst	orc_create_staging.strct
-PREHOOK: query: -- Since complex types are not supported, this query should not vectorize.
-EXPLAIN
-SELECT * FROM orc_create_complex
-PREHOOK: type: QUERY
-POSTHOOK: query: -- Since complex types are not supported, this query should not vectorize.
-EXPLAIN
-SELECT * FROM orc_create_complex
-POSTHOOK: type: QUERY
-Explain
-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: orc_create_complex
-            Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: str (type: string), mp (type: map<string,string>), lst (type: array<string>), strct (type: struct<a:string,b:string>)
-              outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-              File Output Operator
-                compressed: false
-                Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-                table:
-                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
-  Stage: Stage-0
-    Fetch Operator
-      limit: -1
-      Processor Tree:
-        ListSink
-
+POSTHOOK: Lineage: orc_create_complex.val SIMPLE []
+orc_create_staging.str	orc_create_staging.mp	orc_create_staging.lst	orc_create_staging.strct	c1
 PREHOOK: query: SELECT * FROM orc_create_complex
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_complex
@@ -107,129 +74,156 @@ POSTHOOK: query: SELECT * FROM orc_create_complex
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-orc_create_complex.str	orc_create_complex.mp	orc_create_complex.lst	orc_create_complex.strct
-line1	{"key13":"value13","key11":"value11","key12":"value12"}	["a","b","c"]	{"a":"one","b":"two"}
-line2	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]	{"a":"three","b":"four"}
-line3	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]	{"a":"five","b":"six"}
-PREHOOK: query: -- However, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT COUNT(*) FROM orc_create_complex
+orc_create_complex.str	orc_create_complex.mp	orc_create_complex.lst	orc_create_complex.strct	orc_create_complex.val
+line1	{"key13":"value13","key11":"value11","key12":"value12"}	["a","b","c"]	{"a":"one","b":"two"}	0
+line2	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]	{"a":"three","b":"four"}	0
+line3	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]	{"a":"five","b":"six"}	0
+PREHOOK: query: SELECT str FROM orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT str FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+str
+line1
+line2
+line3
+PREHOOK: query: SELECT strct, mp, lst FROM orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT strct, mp, lst FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+strct	mp	lst
+{"a":"one","b":"two"}	{"key13":"value13","key11":"value11","key12":"value12"}	["a","b","c"]
+{"a":"three","b":"four"}	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]
+{"a":"five","b":"six"}	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]
+PREHOOK: query: SELECT lst, str FROM orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT lst, str FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+lst	str
+["a","b","c"]	line1
+["d","e","f"]	line2
+["g","h","i"]	line3
+PREHOOK: query: SELECT mp, str FROM orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT mp, str FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+mp	str
+{"key13":"value13","key11":"value11","key12":"value12"}	line1
+{"key21":"value21","key22":"value22","key23":"value23"}	line2
+{"key31":"value31","key32":"value32","key33":"value33"}	line3
+PREHOOK: query: SELECT strct, str FROM orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT strct, str FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+strct	str
+{"a":"one","b":"two"}	line1
+{"a":"three","b":"four"}	line2
+{"a":"five","b":"six"}	line3
+PREHOOK: query: SELECT strct.B, str FROM orc_create_complex
 PREHOOK: type: QUERY
-POSTHOOK: query: -- However, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT COUNT(*) FROM orc_create_complex
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT strct.B, str FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+b	str
+two	line1
+four	line2
+six	line3
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_0]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: INSERT INTO TABLE orc_create_complex
+SELECT orc_create_staging.*, src1.key FROM orc_create_staging cross join src src1 cross join orc_create_staging spam1 cross join orc_create_staging spam2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_staging
+PREHOOK: Input: default@src
+PREHOOK: Output: default@orc_create_complex
+POSTHOOK: query: INSERT INTO TABLE orc_create_complex
+SELECT orc_create_staging.*, src1.key FROM orc_create_staging cross join src src1 cross join orc_create_staging spam1 cross join orc_create_staging spam2
 POSTHOOK: type: QUERY
-Explain
-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: orc_create_complex
-            Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: COMPLETE
-              Group By Operator
-                aggregations: count()
-                mode: hash
-                outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  sort order: 
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: bigint)
-      Execution mode: vectorized
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations: count(VALUE._col0)
-          mode: mergepartial
-          outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-            table:
-                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
-  Stage: Stage-0
-    Fetch Operator
-      limit: -1
-      Processor Tree:
-        ListSink
-
-PREHOOK: query: SELECT COUNT(*) FROM orc_create_complex
+POSTHOOK: Input: default@orc_create_staging
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@orc_create_complex
+POSTHOOK: Lineage: orc_create_complex.lst SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:lst, type:array<string>, comment:null), ]
+POSTHOOK: Lineage: orc_create_complex.mp SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:mp, type:map<string,string>, comment:null), ]
+POSTHOOK: Lineage: orc_create_complex.str SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:str, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_create_complex.strct SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:strct, type:struct<A:string,B:string>, comment:null), ]
+POSTHOOK: Lineage: orc_create_complex.val SIMPLE [(src)src1.FieldSchema(name:key, type:string, comment:default), ]
+orc_create_staging.str	orc_create_staging.mp	orc_create_staging.lst	orc_create_staging.strct	src1.key
+PREHOOK: query: select count(*) from orc_create_complex
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT COUNT(*) FROM orc_create_complex
+POSTHOOK: query: select count(*) from orc_create_complex
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
 c0
-3
-PREHOOK: query: -- Also, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT str FROM orc_create_complex ORDER BY str
+13503
+PREHOOK: query: SELECT distinct lst, strct FROM orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT distinct lst, strct FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+lst	strct
+["a","b","c"]	{"a":"one","b":"two"}
+["d","e","f"]	{"a":"three","b":"four"}
+["g","h","i"]	{"a":"five","b":"six"}
+PREHOOK: query: SELECT str, count(val)  FROM orc_create_complex GROUP BY str
 PREHOOK: type: QUERY
-POSTHOOK: query: -- Also, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT str FROM orc_create_complex ORDER BY str
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT str, count(val)  FROM orc_create_complex GROUP BY str
 POSTHOOK: type: QUERY
-Explain
-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: orc_create_complex
-            Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: str (type: string)
-              outputColumnNames: _col0
-              Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-      Execution mode: vectorized
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: string)
-          outputColumnNames: _col0
-          Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-            table:
-                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
-  Stage: Stage-0
-    Fetch Operator
-      limit: -1
-      Processor Tree:
-        ListSink
-
-PREHOOK: query: SELECT str FROM orc_create_complex ORDER BY str
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+str	c1
+line1	4501
+line2	4501
+line3	4501
+PREHOOK: query: SELECT strct.B, count(val) FROM orc_create_complex GROUP BY strct.B
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT str FROM orc_create_complex ORDER BY str
+POSTHOOK: query: SELECT strct.B, count(val) FROM orc_create_complex GROUP BY strct.B
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-str
-line1
-line2
-line3
+strct.b	_c1
+four	4501
+six	4501
+two	4501
+PREHOOK: query: SELECT strct, mp, lst, str, count(val) FROM orc_create_complex GROUP BY strct, mp, lst, str
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT strct, mp, lst, str, count(val) FROM orc_create_complex GROUP BY strct, mp, lst, str
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+strct	mp	lst	str	c4
+{"a":"one","b":"two"}	{"key11":"value11","key12":"value12","key13":"value13"}	["a","b","c"]	line1	4501
+{"a":"three","b":"four"}	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]	line2	4501
+{"a":"five","b":"six"}	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]	line3	4501

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/storage-api/src/java/org/apache/hadoop/hive/common/io/encoded/EncodedColumnBatch.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/io/encoded/EncodedColumnBatch.java b/storage-api/src/java/org/apache/hadoop/hive/common/io/encoded/EncodedColumnBatch.java
index 907181e..13772c9 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/common/io/encoded/EncodedColumnBatch.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/io/encoded/EncodedColumnBatch.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.common.io.encoded;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -76,14 +77,17 @@ public class EncodedColumnBatch<BatchKey> {
   /** The key that is used to map this batch to source location. */
   protected BatchKey batchKey;
   /**
-   * Stream data for each stream, for each included column.
-   * For each column, streams are indexed by kind, with missing elements being null.
+   * Stream data for each column that has true in the corresponding hasData position.
+   * For each column, streams are indexed by kind (for ORC), with missing elements being null.
    */
   protected ColumnStreamData[][] columnData;
-  /** Column indexes included in the batch. Correspond to columnData elements. */
-  protected int[] columnIxs;
+  /** Indicates which columns have data. Correspond to columnData elements. */
+  protected boolean[] hasData;
 
   public void reset() {
+    if (hasData != null) {
+      Arrays.fill(hasData, false);
+    }
     if (columnData == null) return;
     for (int i = 0; i < columnData.length; ++i) {
       if (columnData[i] == null) continue;
@@ -93,37 +97,37 @@ public class EncodedColumnBatch<BatchKey> {
     }
   }
 
-  public void initColumn(int colIxMod, int colIx, int streamCount) {
-    columnIxs[colIxMod] = colIx;
-    if (columnData[colIxMod] == null || columnData[colIxMod].length != streamCount) {
-      columnData[colIxMod] = new ColumnStreamData[streamCount];
+  public void initColumn(int colIx, int streamCount) {
+    hasData[colIx] = true;
+    if (columnData[colIx] == null || columnData[colIx].length != streamCount) {
+      columnData[colIx] = new ColumnStreamData[streamCount];
     }
   }
 
-  public void setStreamData(int colIxMod, int streamKind, ColumnStreamData csd) {
-    columnData[colIxMod][streamKind] = csd;
-  }
-
-  public void setAllStreamsData(int colIxMod, int colIx, ColumnStreamData[] sbs) {
-    columnIxs[colIxMod] = colIx;
-    columnData[colIxMod] = sbs;
+  public void setStreamData(int colIx, int streamIx, ColumnStreamData csd) {
+    assert hasData[colIx];
+    columnData[colIx][streamIx] = csd;
   }
 
   public BatchKey getBatchKey() {
     return batchKey;
   }
 
-  public ColumnStreamData[][] getColumnData() {
-    return columnData;
+  public ColumnStreamData[] getColumnData(int colIx) {
+    if (!hasData[colIx]) throw new AssertionError("No data for column " + colIx);
+    return columnData[colIx];
   }
 
-  public int[] getColumnIxs() {
-    return columnIxs;
+  public int getTotalColCount() {
+    return columnData.length; // Includes the columns that have no data
   }
 
   protected void resetColumnArrays(int columnCount) {
-    if (columnIxs != null && columnCount == columnIxs.length) return;
-    columnIxs = new int[columnCount];
+    if (hasData != null && columnCount == hasData.length) {
+      Arrays.fill(hasData, false);
+      return;
+    }
+    hasData = new boolean[columnCount];
     ColumnStreamData[][] columnData = new ColumnStreamData[columnCount][];
     if (this.columnData != null) {
       for (int i = 0; i < Math.min(columnData.length, this.columnData.length); ++i) {
@@ -132,4 +136,8 @@ public class EncodedColumnBatch<BatchKey> {
     }
     this.columnData = columnData;
   }
-}
\ No newline at end of file
+
+  public boolean hasData(int colIx) {
+    return hasData[colIx];
+  }
+}


[32/35] hive git commit: HIVE-15211: Provide support for complex expressions in ON clauses for INNER joins (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-15211: Provide support for complex expressions in ON clauses for INNER joins (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: 893b2553a8f79e600dd6f1dbee48fbbe0b40b58b
Parents: d94ebe8
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Nov 15 21:48:08 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Mon Nov 21 18:51:58 2016 +0000

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |    8 +-
 .../JoinCondTypeCheckProcFactory.java           |  116 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    5 +-
 .../apache/hadoop/hive/ql/parse/QBJoinTree.java |    2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   78 +-
 ql/src/test/queries/clientnegative/join45.q     |   13 +
 ql/src/test/queries/clientpositive/join45.q     |  203 ++
 ql/src/test/results/clientnegative/join45.q.out |   13 +
 ql/src/test/results/clientpositive/join45.q.out | 1771 ++++++++++++++++++
 9 files changed, 2057 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/893b2553/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index 97fcd55..b62df35 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -27,7 +27,6 @@ import java.util.regex.Pattern;
 import org.antlr.runtime.tree.Tree;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.ASTNodeOrigin;
-import org.apache.hadoop.hive.ql.parse.SemanticException;
 
 /**
  * List of all error messages.
@@ -70,9 +69,9 @@ public enum ErrorMsg {
   INVALID_ARGUMENT(10014, "Wrong arguments"),
   INVALID_ARGUMENT_LENGTH(10015, "Arguments length mismatch", "21000"),
   INVALID_ARGUMENT_TYPE(10016, "Argument type mismatch"),
-  INVALID_JOIN_CONDITION_1(10017, "Both left and right aliases encountered in JOIN"),
-  INVALID_JOIN_CONDITION_2(10018, "Neither left nor right aliases encountered in JOIN"),
-  INVALID_JOIN_CONDITION_3(10019, "OR not supported in JOIN currently"),
+  @Deprecated INVALID_JOIN_CONDITION_1(10017, "Both left and right aliases encountered in JOIN"),
+  @Deprecated INVALID_JOIN_CONDITION_2(10018, "Neither left nor right aliases encountered in JOIN"),
+  @Deprecated INVALID_JOIN_CONDITION_3(10019, "OR not supported in JOIN currently"),
   INVALID_TRANSFORM(10020, "TRANSFORM with other SELECT columns not supported"),
   UNSUPPORTED_MULTIPLE_DISTINCTS(10022, "DISTINCT on different columns not supported" +
       " with skew in data"),
@@ -460,6 +459,7 @@ public enum ErrorMsg {
     "requires \"AND <boolean>\" on the 1st WHEN MATCHED clause of <{0}>", true),
   MERGE_TOO_MANY_DELETE(10405, "MERGE statment can have at most 1 WHEN MATCHED ... DELETE clause: <{0}>", true),
   MERGE_TOO_MANY_UPDATE(10406, "MERGE statment can have at most 1 WHEN MATCHED ... UPDATE clause: <{0}>", true),
+  INVALID_JOIN_CONDITION(10407, "Complex condition not supported for (LEFT|RIGHT|FULL) OUTER JOIN"),
   //========================== 20000 range starts here ========================//
   SCRIPT_INIT_ERROR(20000, "Unable to initialize custom script."),
   SCRIPT_IO_ERROR(20001, "An error occurred while reading or writing to your custom script. "

http://git-wip-us.apache.org/repos/asf/hive/blob/893b2553/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinCondTypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinCondTypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinCondTypeCheckProcFactory.java
index 9128d81..cf665ee 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinCondTypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinCondTypeCheckProcFactory.java
@@ -18,17 +18,12 @@
 package org.apache.hadoop.hive.ql.optimizer.calcite.translator;
 
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.Stack;
 
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
-import org.apache.hadoop.hive.ql.exec.FunctionInfo;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
@@ -41,11 +36,6 @@ import org.apache.hadoop.hive.ql.parse.TypeCheckProcFactory;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseCompare;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
 
 /**
  * JoinCondTypeCheckProcFactory is used by Calcite planner(CBO) to generate Join Conditions from Join Condition AST.
@@ -117,7 +107,7 @@ public class JoinCondTypeCheckProcFactory extends TypeCheckProcFactory {
       }
 
       if (tblAliasCnt > 1) {
-        throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_1.getMsg(expr));
+        throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_OR_COLUMN.getMsg(expr));
       }
 
       return (tblAliasCnt == 1) ? true : false;
@@ -132,7 +122,7 @@ public class JoinCondTypeCheckProcFactory extends TypeCheckProcFactory {
         tmp = rr.get(tabName, colAlias);
         if (tmp != null) {
           if (cInfoToRet != null) {
-            throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_1.getMsg(expr));
+            throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_OR_COLUMN.getMsg(expr));
           }
           cInfoToRet = tmp;
         }
@@ -194,7 +184,7 @@ public class JoinCondTypeCheckProcFactory extends TypeCheckProcFactory {
         tmp = rr.get(tabName, colAlias);
         if (tmp != null) {
           if (cInfoToRet != null) {
-            throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_1.getMsg(expr));
+            throw new SemanticException(ErrorMsg.AMBIGUOUS_TABLE_OR_COLUMN.getMsg(expr));
           }
           cInfoToRet = tmp;
         }
@@ -202,106 +192,6 @@ public class JoinCondTypeCheckProcFactory extends TypeCheckProcFactory {
 
       return cInfoToRet;
     }
-
-    @Override
-    protected void validateUDF(ASTNode expr, boolean isFunction, TypeCheckCtx ctx, FunctionInfo fi,
-        List<ExprNodeDesc> children, GenericUDF genericUDF) throws SemanticException {
-      super.validateUDF(expr, isFunction, ctx, fi, children, genericUDF);
-
-      JoinTypeCheckCtx jCtx = (JoinTypeCheckCtx) ctx;
-
-      // Join Condition can not contain disjunctions
-      if (genericUDF instanceof GenericUDFOPOr) {
-        throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_3.getMsg(expr));
-      }
-
-      // Non Conjunctive elements have further limitations in Join conditions
-      if (!(genericUDF instanceof GenericUDFOPAnd)) {
-        // Non Comparison UDF other than 'and' can not use inputs from both side
-        if (!(genericUDF instanceof GenericUDFBaseCompare)) {
-          if (genericUDFargsRefersToBothInput(genericUDF, children, jCtx.getInputRRList())) {
-            throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_1.getMsg(expr));
-          }
-        } else if (genericUDF instanceof GenericUDFBaseCompare) {
-          // Comparisons of non literals LHS/RHS can not refer to inputs from
-          // both sides
-          if (children.size() == 2 && !(children.get(0) instanceof ExprNodeConstantDesc)
-              && !(children.get(1) instanceof ExprNodeConstantDesc)) {
-            if (comparisonUDFargsRefersToBothInput((GenericUDFBaseCompare) genericUDF, children,
-                jCtx.getInputRRList())) {
-              throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_1.getMsg(expr));
-            }
-          }
-        }
-      }
-    }
-
-    private static boolean genericUDFargsRefersToBothInput(GenericUDF udf,
-        List<ExprNodeDesc> children, List<RowResolver> inputRRList) {
-      boolean argsRefersToBothInput = false;
-
-      Map<Integer, ExprNodeDesc> hasCodeToColDescMap = new HashMap<Integer, ExprNodeDesc>();
-      for (ExprNodeDesc child : children) {
-        ExprNodeDescUtils.getExprNodeColumnDesc(child, hasCodeToColDescMap);
-      }
-      Set<Integer> inputRef = getInputRef(hasCodeToColDescMap.values(), inputRRList);
-
-      if (inputRef.size() > 1)
-        argsRefersToBothInput = true;
-
-      return argsRefersToBothInput;
-    }
-
-    private static boolean comparisonUDFargsRefersToBothInput(GenericUDFBaseCompare comparisonUDF,
-        List<ExprNodeDesc> children, List<RowResolver> inputRRList) {
-      boolean argsRefersToBothInput = false;
-
-      Map<Integer, ExprNodeDesc> lhsHashCodeToColDescMap = new HashMap<Integer, ExprNodeDesc>();
-      Map<Integer, ExprNodeDesc> rhsHashCodeToColDescMap = new HashMap<Integer, ExprNodeDesc>();
-      ExprNodeDescUtils.getExprNodeColumnDesc(children.get(0), lhsHashCodeToColDescMap);
-      ExprNodeDescUtils.getExprNodeColumnDesc(children.get(1), rhsHashCodeToColDescMap);
-      Set<Integer> lhsInputRef = getInputRef(lhsHashCodeToColDescMap.values(), inputRRList);
-      Set<Integer> rhsInputRef = getInputRef(rhsHashCodeToColDescMap.values(), inputRRList);
-
-      if (lhsInputRef.size() > 1 || rhsInputRef.size() > 1)
-        argsRefersToBothInput = true;
-
-      return argsRefersToBothInput;
-    }
-
-    private static Set<Integer> getInputRef(Collection<ExprNodeDesc> colDescSet,
-        List<RowResolver> inputRRList) {
-      String tableAlias;
-      RowResolver inputRR;
-      Set<Integer> inputLineage = new HashSet<Integer>();
-
-      for (ExprNodeDesc col : colDescSet) {
-        ExprNodeColumnDesc colDesc = (ExprNodeColumnDesc) col;
-        tableAlias = colDesc.getTabAlias();
-
-        for (int i = 0; i < inputRRList.size(); i++) {
-          inputRR = inputRRList.get(i);
-
-          // If table Alias is present check if InputRR has that table and then
-          // check for internal name
-          // else if table alias is null then check with internal name in all
-          // inputRR.
-          if (tableAlias != null) {
-            if (inputRR.hasTableAlias(tableAlias)) {
-              if (inputRR.doesInvRslvMapContain(colDesc.getColumn())) {
-                inputLineage.add(i);
-              }
-            }
-          } else {
-            if (inputRR.doesInvRslvMapContain(colDesc.getColumn())) {
-              inputLineage.add(i);
-            }
-          }
-        }
-      }
-
-      return inputLineage;
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/893b2553/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 78011c2..6965f8f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -1543,12 +1543,11 @@ public class CalcitePlanner extends SemanticAnalyzer {
         JoinTypeCheckCtx jCtx = new JoinTypeCheckCtx(leftRR, rightRR, hiveJoinType);
         Map<ASTNode, ExprNodeDesc> exprNodes = JoinCondTypeCheckProcFactory.genExprNode(joinCond,
             jCtx);
-        if (jCtx.getError() != null)
+        if (jCtx.getError() != null) {
           throw new SemanticException(SemanticAnalyzer.generateErrorMessage(jCtx.getErrorSrcNode(),
               jCtx.getError()));
-
+        }
         ExprNodeDesc joinCondnExprNode = exprNodes.get(joinCond);
-
         List<RelNode> inputRels = new ArrayList<RelNode>();
         inputRels.add(leftRel);
         inputRels.add(rightRel);

http://git-wip-us.apache.org/repos/asf/hive/blob/893b2553/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java
index a3e95ce..ec76fb7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/QBJoinTree.java
@@ -420,7 +420,7 @@ public class QBJoinTree implements Serializable, Cloneable {
 
     // clone postJoinFilters
     for (ASTNode filter : postJoinFilters) {
-      cloned.getPostJoinFilters().add(filter);
+      cloned.addPostJoinFilter(filter);
     }
     // clone rhsSemijoin
     for (Entry<String, ArrayList<ASTNode>> entry : rhsSemijoin.entrySet()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/893b2553/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 7d8b2bd..3bc6fe4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -65,7 +65,6 @@ import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -2581,8 +2580,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       List<String> rightAliases, ASTNode condn, QBJoinTree joinTree,
       List<String> leftSrc) throws SemanticException {
     if ((leftAliases.size() != 0) && (rightAliases.size() != 0)) {
-      throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_1
-          .getMsg(condn));
+      joinTree.addPostJoinFilter(condn);
+      return;
     }
 
     if (rightAliases.size() != 0) {
@@ -2596,8 +2595,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         }
       }
     } else {
-      throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_2
-          .getMsg(condn));
+      joinTree.addPostJoinFilter(condn);
     }
   }
 
@@ -2791,8 +2789,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     switch (joinCond.getToken().getType()) {
     case HiveParser.KW_OR:
-      throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_3
-          .getMsg(joinCond));
+      joinTree.addPostJoinFilter(joinCond);
+      break;
 
     case HiveParser.KW_AND:
       parseJoinCondition(joinTree, (ASTNode) joinCond.getChild(0), leftSrc, type, aliasToOpInfo);
@@ -2821,15 +2819,13 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       // * join is right outer and filter is on right alias
       if (((leftCondAl1.size() != 0) && (leftCondAl2.size() != 0))
           || ((rightCondAl1.size() != 0) && (rightCondAl2.size() != 0))) {
-        throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_1
-            .getMsg(joinCond));
+        joinTree.addPostJoinFilter(joinCond);
+      } else {
+        applyEqualityPredicateToQBJoinTree(joinTree, type, leftSrc,
+            joinCond, leftCondn, rightCondn,
+            leftCondAl1, leftCondAl2,
+            rightCondAl1, rightCondAl2);
       }
-
-      applyEqualityPredicateToQBJoinTree(joinTree, type, leftSrc,
-          joinCond, leftCondn, rightCondn,
-          leftCondAl1, leftCondAl2,
-          rightCondAl1, rightCondAl2);
-
       break;
 
     default:
@@ -2871,23 +2867,22 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       }
 
       if (!leftAliasNull && !rightAliasNull) {
-        throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION_1
-            .getMsg(joinCond));
-      }
-
-      if (!leftAliasNull) {
-        if (type.equals(JoinType.LEFTOUTER)
-            || type.equals(JoinType.FULLOUTER)) {
-            joinTree.getFilters().get(0).add(joinCond);
-        } else {
-          joinTree.getFiltersForPushing().get(0).add(joinCond);
-        }
+        joinTree.addPostJoinFilter(joinCond);
       } else {
-        if (type.equals(JoinType.RIGHTOUTER)
-            || type.equals(JoinType.FULLOUTER)) {
-            joinTree.getFilters().get(1).add(joinCond);
+        if (!leftAliasNull) {
+          if (type.equals(JoinType.LEFTOUTER)
+              || type.equals(JoinType.FULLOUTER)) {
+              joinTree.getFilters().get(0).add(joinCond);
+          } else {
+            joinTree.getFiltersForPushing().get(0).add(joinCond);
+          }
         } else {
-          joinTree.getFiltersForPushing().get(1).add(joinCond);
+          if (type.equals(JoinType.RIGHTOUTER)
+              || type.equals(JoinType.FULLOUTER)) {
+              joinTree.getFilters().get(1).add(joinCond);
+          } else {
+            joinTree.getFiltersForPushing().get(1).add(joinCond);
+          }
         }
       }
 
@@ -8107,9 +8102,16 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     joinOp.getConf().setQBJoinTreeProps(joinTree);
     joinContext.put(joinOp, joinTree);
 
+    // Safety check for postconditions; currently we do not support them for outer join
+    if (joinTree.getPostJoinFilters().size() != 0 && !joinTree.getNoOuterJoin()) {
+      throw new SemanticException(ErrorMsg.INVALID_JOIN_CONDITION.getMsg());
+    }
     Operator op = joinOp;
-    for(ASTNode condn : joinTree.getPostJoinFilters() ) {
+    for(ASTNode condn : joinTree.getPostJoinFilters()) {
       op = genFilterPlan(qb, condn, op, false);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Generated " + op + " with post-filtering conditions after JOIN operator");
+      }
     }
     return op;
   }
@@ -8928,6 +8930,15 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       }
       target.setMapAliases(mapAliases);
     }
+
+    if (node.getPostJoinFilters().size() != 0) {
+      // Safety check: if we are merging join operators and there are post-filtering
+      // conditions, they cannot be outer joins
+      assert node.getNoOuterJoin() && target.getNoOuterJoin();
+      for (ASTNode exprPostFilter : node.getPostJoinFilters()) {
+        target.addPostJoinFilter(exprPostFilter);
+      }
+    }
   }
 
   private ObjectPair<Integer, int[]> findMergePos(QBJoinTree node, QBJoinTree target) {
@@ -9035,6 +9046,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         if (prevType != null && prevType != currType) {
           break;
         }
+        if ((!node.getNoOuterJoin() && node.getPostJoinFilters().size() != 0) ||
+                (!target.getNoOuterJoin() && target.getPostJoinFilters().size() != 0)) {
+          // Outer joins with post-filtering conditions cannot be merged
+          break;
+        }
         ObjectPair<Integer, int[]> mergeDetails = findMergePos(node, target);
         int pos = mergeDetails.getFirst();
         if (pos >= 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/893b2553/ql/src/test/queries/clientnegative/join45.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/join45.q b/ql/src/test/queries/clientnegative/join45.q
new file mode 100644
index 0000000..4e8db96
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/join45.q
@@ -0,0 +1,13 @@
+set hive.strict.checks.cartesian.product=false;
+
+-- SORT_QUERY_RESULTS
+
+CREATE TABLE mytable(val1 INT, val2 INT, val3 INT);
+
+-- Outer join with complex pred: not supported
+EXPLAIN
+SELECT *
+FROM mytable src1 LEFT OUTER JOIN mytable src2
+ON (src1.val1+src2.val1>= 2450816
+  AND src1.val1+src2.val1<= 2451500);
+

http://git-wip-us.apache.org/repos/asf/hive/blob/893b2553/ql/src/test/queries/clientpositive/join45.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/join45.q b/ql/src/test/queries/clientpositive/join45.q
new file mode 100644
index 0000000..54e422d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/join45.q
@@ -0,0 +1,203 @@
+set hive.strict.checks.cartesian.product=false;
+
+-- Conjunction with pred on multiple inputs and single inputs
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  AND src1.value between 100 and 102
+  AND src.value between 100 and 102)
+LIMIT 10;
+
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  AND src1.value between 100 and 102
+  AND src.value between 100 and 102)
+LIMIT 10;
+
+-- Conjunction with pred on multiple inputs and none
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key AND true)
+LIMIT 10;
+
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key AND true)
+LIMIT 10;
+
+-- Conjunction with pred on single inputs and none
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.value between 100 and 102
+  AND src.value between 100 and 102
+  AND true)
+LIMIT 10;
+
+SELECT *
+FROM src1 JOIN src
+ON (src1.value between 100 and 102
+  AND src.value between 100 and 102
+  AND true)
+LIMIT 10;
+
+-- Disjunction with pred on multiple inputs and single inputs
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  OR src1.value between 100 and 102
+  OR src.value between 100 and 102)
+LIMIT 10;
+
+SELECT *
+FROM src1 JOIN src
+ON (src1.key=src.key
+  OR src1.value between 100 and 102
+  OR src.value between 100 and 102)
+LIMIT 10;
+
+-- Conjunction with multiple inputs on one side
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  AND src1.key+src.key <= 102)
+LIMIT 10;
+
+SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  AND src1.key+src.key <= 102)
+LIMIT 10;
+
+-- Disjunction with multiple inputs on one side
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  OR src1.key+src.key <= 102)
+LIMIT 10;
+
+SELECT *
+FROM src1 JOIN src
+ON (src1.key+src.key >= 100
+  OR src1.key+src.key <= 102)
+LIMIT 10;
+
+-- Function with multiple inputs on one side
+EXPLAIN
+SELECT *
+FROM src1 JOIN src
+ON ((src1.key,src.key) IN ((100,100),(101,101),(102,102)))
+LIMIT 10;
+
+SELECT *
+FROM src1 JOIN src
+ON ((src1.key,src.key) IN ((100,100),(101,101),(102,102)))
+LIMIT 10;
+
+-- Chained 1
+EXPLAIN
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+LEFT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10;
+
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+LEFT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10;
+
+-- Chained 2
+EXPLAIN
+SELECT *
+FROM src
+LEFT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10;
+
+SELECT *
+FROM src
+LEFT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10;
+
+-- Chained 3
+EXPLAIN
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+RIGHT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10;
+
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+RIGHT OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10;
+
+-- Chained 4
+EXPLAIN
+SELECT *
+FROM src
+RIGHT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10;
+
+SELECT *
+FROM src
+RIGHT OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10;
+
+-- Chained 5
+EXPLAIN
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+FULL OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10;
+
+SELECT *
+FROM src
+JOIN src1 a ON (a.key+src.key >= 100)
+FULL OUTER JOIN src1 b ON (b.key = src.key)
+LIMIT 10;
+
+-- Chained 6
+EXPLAIN
+SELECT *
+FROM src
+FULL OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10;
+
+SELECT *
+FROM src
+FULL OUTER JOIN src1 a ON (a.key = src.key)
+JOIN src1 b ON (b.key+src.key<= 102)
+LIMIT 10;
+
+-- Right outer join with multiple inner joins and mixed conditions
+EXPLAIN
+SELECT *
+FROM cbo_t1 t1
+RIGHT OUTER JOIN cbo_t1 t2 ON (t2.key = t1.key)
+JOIN cbo_t1 t3 ON (t3.key = t2.key or t3.value = t2.value and t2.c_int = t1.c_int)
+JOIN cbo_t1 t4 ON (t4.key = t2.key or  t2.c_float = t4.c_float and t4.value = t2.value)
+JOIN cbo_t1 t5 ON (t5.key = t2.key or  t2.c_boolean = t4.c_boolean and t5.c_int = 42)
+LIMIT 10;
+
+SELECT *
+FROM cbo_t1 t1
+RIGHT OUTER JOIN cbo_t1 t2 ON (t2.key = t1.key)
+JOIN cbo_t1 t3 ON (t3.key = t2.key or t3.value = t2.value and t2.c_int = t1.c_int)
+JOIN cbo_t1 t4 ON (t4.key = t2.key or  t2.c_float = t4.c_float and t4.value = t2.value)
+JOIN cbo_t1 t5 ON (t5.key = t2.key or  t2.c_boolean = t4.c_boolean and t5.c_int = 42)
+LIMIT 10;

http://git-wip-us.apache.org/repos/asf/hive/blob/893b2553/ql/src/test/results/clientnegative/join45.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/join45.q.out b/ql/src/test/results/clientnegative/join45.q.out
new file mode 100644
index 0000000..87ef769
--- /dev/null
+++ b/ql/src/test/results/clientnegative/join45.q.out
@@ -0,0 +1,13 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+CREATE TABLE mytable(val1 INT, val2 INT, val3 INT)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@mytable
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+CREATE TABLE mytable(val1 INT, val2 INT, val3 INT)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@mytable
+FAILED: SemanticException [Error 10407]: Complex condition not supported for (LEFT|RIGHT|FULL) OUTER JOIN


[11/35] hive git commit: HIVE-14815: Implement Parquet vectorization reader for Primitive types(Ferdinand Xu, review by Chao Sun) This closes #104

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedParquetRecordReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedParquetRecordReader.java
new file mode 100644
index 0000000..f94c49a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedParquetRecordReader.java
@@ -0,0 +1,289 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.io.parquet.vector;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import org.apache.hadoop.hive.ql.io.IOConstants;
+import org.apache.hadoop.hive.ql.io.parquet.ParquetRecordReaderBase;
+import org.apache.hadoop.hive.ql.io.parquet.ProjectionPusher;
+import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.hive.serde2.SerDeStats;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetInputSplit;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups;
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.range;
+import static org.apache.parquet.hadoop.ParquetFileReader.readFooter;
+import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter;
+
+/**
+ * This reader is used to read a batch of record from inputsplit, part of the code is referred
+ * from Apache Spark and Apache Parquet.
+ */
+public class VectorizedParquetRecordReader extends ParquetRecordReaderBase
+  implements RecordReader<NullWritable, VectorizedRowBatch> {
+  public static final Logger LOG = LoggerFactory.getLogger(VectorizedParquetRecordReader.class);
+
+  private List<Integer> colsToInclude;
+
+  protected MessageType fileSchema;
+  protected MessageType requestedSchema;
+  private List<String> columnNamesList;
+  private List<TypeInfo> columnTypesList;
+  private VectorizedRowBatchCtx rbCtx;
+
+  /**
+   * For each request column, the reader to read this column. This is NULL if this column
+   * is missing from the file, in which case we populate the attribute with NULL.
+   */
+  private VectorizedColumnReader[] columnReaders;
+
+  /**
+   * The number of rows that have been returned.
+   */
+  private long rowsReturned;
+
+  /**
+   * The number of rows that have been reading, including the current in flight row group.
+   */
+  private long totalCountLoadedSoFar = 0;
+
+  /**
+   * The total number of rows this RecordReader will eventually read. The sum of the
+   * rows of all the row groups.
+   */
+  protected long totalRowCount;
+
+  @VisibleForTesting
+  public VectorizedParquetRecordReader(
+    InputSplit inputSplit,
+    JobConf conf) {
+    try {
+      serDeStats = new SerDeStats();
+      projectionPusher = new ProjectionPusher();
+      initialize(inputSplit, conf);
+      colsToInclude = ColumnProjectionUtils.getReadColumnIDs(conf);
+      rbCtx = Utilities.getVectorizedRowBatchCtx(conf);
+    } catch (Throwable e) {
+      LOG.error("Failed to create the vectorized reader due to exception " + e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  public VectorizedParquetRecordReader(
+    org.apache.hadoop.mapred.InputSplit oldInputSplit,
+    JobConf conf) {
+    try {
+      serDeStats = new SerDeStats();
+      projectionPusher = new ProjectionPusher();
+      initialize(getSplit(oldInputSplit, conf), conf);
+      colsToInclude = ColumnProjectionUtils.getReadColumnIDs(conf);
+      rbCtx = Utilities.getVectorizedRowBatchCtx(conf);
+    } catch (Throwable e) {
+      LOG.error("Failed to create the vectorized reader due to exception " + e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void initialize(
+    InputSplit oldSplit,
+    JobConf configuration) throws IOException, InterruptedException {
+    jobConf = configuration;
+    ParquetMetadata footer;
+    List<BlockMetaData> blocks;
+    ParquetInputSplit split = (ParquetInputSplit) oldSplit;
+    boolean indexAccess =
+      configuration.getBoolean(DataWritableReadSupport.PARQUET_COLUMN_INDEX_ACCESS, false);
+    this.file = split.getPath();
+    long[] rowGroupOffsets = split.getRowGroupOffsets();
+
+    String columnNames = configuration.get(IOConstants.COLUMNS);
+    columnNamesList = DataWritableReadSupport.getColumnNames(columnNames);
+    String columnTypes = configuration.get(IOConstants.COLUMNS_TYPES);
+    columnTypesList = DataWritableReadSupport.getColumnTypes(columnTypes);
+
+    // if task.side.metadata is set, rowGroupOffsets is null
+    if (rowGroupOffsets == null) {
+      //TODO check whether rowGroupOffSets can be null
+      // then we need to apply the predicate push down filter
+      footer = readFooter(configuration, file, range(split.getStart(), split.getEnd()));
+      MessageType fileSchema = footer.getFileMetaData().getSchema();
+      FilterCompat.Filter filter = getFilter(configuration);
+      blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema);
+    } else {
+      // otherwise we find the row groups that were selected on the client
+      footer = readFooter(configuration, file, NO_FILTER);
+      Set<Long> offsets = new HashSet<>();
+      for (long offset : rowGroupOffsets) {
+        offsets.add(offset);
+      }
+      blocks = new ArrayList<>();
+      for (BlockMetaData block : footer.getBlocks()) {
+        if (offsets.contains(block.getStartingPos())) {
+          blocks.add(block);
+        }
+      }
+      // verify we found them all
+      if (blocks.size() != rowGroupOffsets.length) {
+        long[] foundRowGroupOffsets = new long[footer.getBlocks().size()];
+        for (int i = 0; i < foundRowGroupOffsets.length; i++) {
+          foundRowGroupOffsets[i] = footer.getBlocks().get(i).getStartingPos();
+        }
+        // this should never happen.
+        // provide a good error message in case there's a bug
+        throw new IllegalStateException(
+          "All the offsets listed in the split should be found in the file."
+            + " expected: " + Arrays.toString(rowGroupOffsets)
+            + " found: " + blocks
+            + " out of: " + Arrays.toString(foundRowGroupOffsets)
+            + " in range " + split.getStart() + ", " + split.getEnd());
+      }
+    }
+
+    for (BlockMetaData block : blocks) {
+      this.totalRowCount += block.getRowCount();
+    }
+    this.fileSchema = footer.getFileMetaData().getSchema();
+
+    MessageType tableSchema;
+    if (indexAccess) {
+      List<Integer> indexSequence = new ArrayList<>();
+
+      // Generates a sequence list of indexes
+      for(int i = 0; i < columnNamesList.size(); i++) {
+        indexSequence.add(i);
+      }
+
+      tableSchema = DataWritableReadSupport.getSchemaByIndex(fileSchema, columnNamesList,
+        indexSequence);
+    } else {
+      tableSchema = DataWritableReadSupport.getSchemaByName(fileSchema, columnNamesList,
+        columnTypesList);
+    }
+
+    List<Integer> indexColumnsWanted = ColumnProjectionUtils.getReadColumnIDs(configuration);
+    if (!ColumnProjectionUtils.isReadAllColumns(configuration) && !indexColumnsWanted.isEmpty()) {
+      requestedSchema =
+        DataWritableReadSupport.getSchemaByIndex(tableSchema, columnNamesList, indexColumnsWanted);
+    } else {
+      requestedSchema = fileSchema;
+    }
+
+    this.reader = new ParquetFileReader(
+      configuration, footer.getFileMetaData(), file, blocks, requestedSchema.getColumns());
+  }
+
+  @Override
+  public boolean next(
+    NullWritable nullWritable,
+    VectorizedRowBatch vectorizedRowBatch) throws IOException {
+    return nextBatch(vectorizedRowBatch);
+  }
+
+  @Override
+  public NullWritable createKey() {
+    return NullWritable.get();
+  }
+
+  @Override
+  public VectorizedRowBatch createValue() {
+    return rbCtx.createVectorizedRowBatch();
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    //TODO
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public float getProgress() throws IOException {
+    //TODO
+    return 0;
+  }
+
+  /**
+   * Advances to the next batch of rows. Returns false if there are no more.
+   */
+  private boolean nextBatch(VectorizedRowBatch columnarBatch) throws IOException {
+    columnarBatch.reset();
+    if (rowsReturned >= totalRowCount) {
+      return false;
+    }
+    checkEndOfRowGroup();
+
+    int num = (int) Math.min(VectorizedRowBatch.DEFAULT_SIZE, totalCountLoadedSoFar - rowsReturned);
+    for (int i = 0; i < columnReaders.length; ++i) {
+      if (columnReaders[i] == null) {
+        continue;
+      }
+      columnarBatch.cols[colsToInclude.get(i)].isRepeating = true;
+      columnReaders[i].readBatch(num, columnarBatch.cols[colsToInclude.get(i)],
+        columnTypesList.get(colsToInclude.get(i)));
+    }
+    rowsReturned += num;
+    columnarBatch.size = num;
+    return true;
+  }
+
+  private void checkEndOfRowGroup() throws IOException {
+    if (rowsReturned != totalCountLoadedSoFar) {
+      return;
+    }
+    PageReadStore pages = reader.readNextRowGroup();
+    if (pages == null) {
+      throw new IOException("expecting more rows but reached last block. Read "
+        + rowsReturned + " out of " + totalRowCount);
+    }
+    List<ColumnDescriptor> columns = requestedSchema.getColumns();
+    List<Type> types = requestedSchema.getFields();
+    columnReaders = new VectorizedColumnReader[columns.size()];
+    for (int i = 0; i < columns.size(); ++i) {
+      columnReaders[i] =
+        new VectorizedColumnReader(columns.get(i), pages.getPageReader(columns.get(i)),
+          skipTimestampConversion, types.get(i));
+    }
+    totalCountLoadedSoFar += pages.getRowCount();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestVectorizedColumnReader.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestVectorizedColumnReader.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestVectorizedColumnReader.java
new file mode 100644
index 0000000..276ff19
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestVectorizedColumnReader.java
@@ -0,0 +1,429 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io.parquet;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import org.apache.hadoop.hive.ql.io.IOConstants;
+import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport;
+import org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector;
+import org.apache.hadoop.hive.ql.io.parquet.vector.VectorizedParquetRecordReader;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroupFactory;
+import org.apache.parquet.hadoop.ParquetInputFormat;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.MessageType;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import static junit.framework.Assert.assertTrue;
+import static junit.framework.TestCase.assertFalse;
+import static org.apache.parquet.column.ParquetProperties.WriterVersion.PARQUET_1_0;
+import static org.apache.parquet.hadoop.api.ReadSupport.PARQUET_READ_SCHEMA;
+import static org.apache.parquet.hadoop.metadata.CompressionCodecName.GZIP;
+import static org.apache.parquet.schema.MessageTypeParser.parseMessageType;
+import static org.junit.Assert.assertEquals;
+
+public class TestVectorizedColumnReader {
+
+  private static final int nElements = 2500;
+  protected static final Configuration conf = new Configuration();
+  protected static final Path file =
+    new Path("target/test/TestParquetVectorReader/testParquetFile");
+  private static String[] uniqueStrs = new String[nElements];
+  private static boolean[] isNulls = new boolean[nElements];
+  private static Random random = new Random();
+  protected static final MessageType schema = parseMessageType(
+    "message test { "
+      + "required int32 int32_field; "
+      + "required int64 int64_field; "
+      + "required int96 int96_field; "
+      + "required double double_field; "
+      + "required float float_field; "
+      + "required boolean boolean_field; "
+      + "required fixed_len_byte_array(3) flba_field; "
+      + "optional fixed_len_byte_array(1) some_null_field; "
+      + "optional fixed_len_byte_array(1) all_null_field; "
+      + "optional binary binary_field; "
+      + "optional binary binary_field_non_repeating; "
+      + "} ");
+
+  @AfterClass
+  public static void cleanup() throws IOException {
+    FileSystem fs = file.getFileSystem(conf);
+    if (fs.exists(file)) {
+      fs.delete(file, true);
+    }
+  }
+
+  @BeforeClass
+  public static void prepareFile() throws IOException {
+    cleanup();
+
+    boolean dictionaryEnabled = true;
+    boolean validating = false;
+    GroupWriteSupport.setSchema(schema, conf);
+    SimpleGroupFactory f = new SimpleGroupFactory(schema);
+    ParquetWriter<Group> writer = new ParquetWriter<Group>(
+      file,
+      new GroupWriteSupport(),
+      GZIP, 1024*1024, 1024, 1024*1024,
+      dictionaryEnabled, validating, PARQUET_1_0, conf);
+    writeData(f, writer);
+  }
+
+  protected static void writeData(SimpleGroupFactory f, ParquetWriter<Group> writer) throws IOException {
+    initialStrings(uniqueStrs);
+    for (int i = 0; i < nElements; i++) {
+      Group group = f.newGroup()
+        .append("int32_field", i)
+        .append("int64_field", (long) 2 * i)
+        .append("int96_field", Binary.fromReusedByteArray("999999999999".getBytes()))
+        .append("double_field", i * 1.0)
+        .append("float_field", ((float) (i * 2.0)))
+        .append("boolean_field", i % 5 == 0)
+        .append("flba_field", "abc");
+
+      if (i % 2 == 1) {
+        group.append("some_null_field", "x");
+      }
+
+      if (i % 13 != 1) {
+        int binaryLen = i % 10;
+        group.append("binary_field",
+          Binary.fromString(new String(new char[binaryLen]).replace("\0", "x")));
+      }
+
+      if (uniqueStrs[i] != null) {
+        group.append("binary_field_non_repeating", Binary.fromString(uniqueStrs[i]));
+      }
+      writer.write(group);
+    }
+    writer.close();
+  }
+
+  private static String getRandomStr() {
+    int len = random.nextInt(10);
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < len; i++) {
+      sb.append((char) ('a' + random.nextInt(25)));
+    }
+    return sb.toString();
+  }
+
+  public static void initialStrings(String[] uniqueStrs) {
+    for (int i = 0; i < uniqueStrs.length; i++) {
+      String str = getRandomStr();
+      if (!str.isEmpty()) {
+        uniqueStrs[i] = str;
+        isNulls[i] = false;
+      }else{
+        isNulls[i] = true;
+      }
+    }
+  }
+
+  private VectorizedParquetRecordReader createParquetReader(String schemaString, Configuration conf)
+    throws IOException, InterruptedException, HiveException {
+    conf.set(PARQUET_READ_SCHEMA, schemaString);
+    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED, true);
+    HiveConf.setVar(conf, HiveConf.ConfVars.PLAN, "//tmp");
+
+    Job vectorJob = new Job(conf, "read vector");
+    ParquetInputFormat.setInputPaths(vectorJob, file);
+    ParquetInputFormat parquetInputFormat = new ParquetInputFormat(GroupReadSupport.class);
+    InputSplit split = (InputSplit) parquetInputFormat.getSplits(vectorJob).get(0);
+    initialVectorizedRowBatchCtx(conf);
+    return new VectorizedParquetRecordReader(split, new JobConf(conf));
+  }
+
+  private void initialVectorizedRowBatchCtx(Configuration conf) throws HiveException {
+    MapWork mapWork = new MapWork();
+    VectorizedRowBatchCtx rbCtx = new VectorizedRowBatchCtx();
+    rbCtx.init(createStructObjectInspector(conf), new String[0]);
+    mapWork.setVectorMode(true);
+    mapWork.setVectorizedRowBatchCtx(rbCtx);
+    Utilities.setMapWork(conf, mapWork);
+  }
+
+  private StructObjectInspector createStructObjectInspector(Configuration conf) {
+    // Create row related objects
+    String columnNames = conf.get(IOConstants.COLUMNS);
+    List<String> columnNamesList = DataWritableReadSupport.getColumnNames(columnNames);
+    String columnTypes = conf.get(IOConstants.COLUMNS_TYPES);
+    List<TypeInfo> columnTypesList = DataWritableReadSupport.getColumnTypes(columnTypes);
+    TypeInfo rowTypeInfo = TypeInfoFactory.getStructTypeInfo(columnNamesList, columnTypesList);
+    return new ArrayWritableObjectInspector((StructTypeInfo) rowTypeInfo);
+  }
+
+  @Test
+  public void testIntRead() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(IOConstants.COLUMNS,"int32_field");
+    conf.set(IOConstants.COLUMNS_TYPES,"int");
+    conf.setBoolean(ColumnProjectionUtils.READ_ALL_COLUMNS, false);
+    conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0");
+    VectorizedParquetRecordReader reader =
+      createParquetReader("message test { required int32 int32_field;}", conf);
+    VectorizedRowBatch previous = reader.createValue();
+    try {
+      long c = 0;
+      while (reader.next(NullWritable.get(), previous)) {
+        LongColumnVector vector = (LongColumnVector) previous.cols[0];
+        assertTrue(vector.noNulls);
+        for (int i = 0; i < vector.vector.length; i++) {
+          if(c == nElements){
+            break;
+          }
+          assertEquals(c, vector.vector[i]);
+          assertFalse(vector.isNull[i]);
+          c++;
+        }
+      }
+      assertEquals(nElements, c);
+    } finally {
+      reader.close();
+    }
+  }
+
+  @Test
+  public void testLongRead() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(IOConstants.COLUMNS,"int64_field");
+    conf.set(IOConstants.COLUMNS_TYPES, "bigint");
+    conf.setBoolean(ColumnProjectionUtils.READ_ALL_COLUMNS, false);
+    conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0");
+    VectorizedParquetRecordReader reader =
+      createParquetReader("message test { required int64 int64_field;}", conf);
+    VectorizedRowBatch previous = reader.createValue();
+    try {
+      long c = 0;
+      while (reader.next(NullWritable.get(), previous)) {
+        LongColumnVector vector = (LongColumnVector) previous.cols[0];
+        assertTrue(vector.noNulls);
+        for (int i = 0; i < vector.vector.length; i++) {
+          if(c == nElements){
+            break;
+          }
+          assertEquals(2 * c, vector.vector[i]);
+          assertFalse(vector.isNull[i]);
+          c++;
+        }
+      }
+      assertEquals(nElements, c);
+    } finally {
+      reader.close();
+    }
+  }
+
+  @Test
+  public void testDoubleRead() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(IOConstants.COLUMNS,"double_field");
+    conf.set(IOConstants.COLUMNS_TYPES, "double");
+    conf.setBoolean(ColumnProjectionUtils.READ_ALL_COLUMNS, false);
+    conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0");
+    VectorizedParquetRecordReader reader =
+      createParquetReader("message test { required double double_field;}", conf);
+    VectorizedRowBatch previous = reader.createValue();
+    try {
+      long c = 0;
+      while (reader.next(NullWritable.get(), previous)) {
+        DoubleColumnVector vector = (DoubleColumnVector) previous.cols[0];
+        assertTrue(vector.noNulls);
+        for (int i = 0; i < vector.vector.length; i++) {
+          if(c == nElements){
+            break;
+          }
+          assertEquals(1.0 * c, vector.vector[i], 0);
+          assertFalse(vector.isNull[i]);
+          c++;
+        }
+      }
+      assertEquals(nElements, c);
+    } finally {
+      reader.close();
+    }
+  }
+
+  @Test
+  public void testFloatRead() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(IOConstants.COLUMNS,"float_field");
+    conf.set(IOConstants.COLUMNS_TYPES, "float");
+    conf.setBoolean(ColumnProjectionUtils.READ_ALL_COLUMNS, false);
+    conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0");
+    VectorizedParquetRecordReader reader =
+      createParquetReader("message test { required float float_field;}", conf);
+    VectorizedRowBatch previous = reader.createValue();
+    try {
+      long c = 0;
+      while (reader.next(NullWritable.get(), previous)) {
+        DoubleColumnVector vector = (DoubleColumnVector) previous.cols[0];
+        assertTrue(vector.noNulls);
+        for (int i = 0; i < vector.vector.length; i++) {
+          if(c == nElements){
+            break;
+          }
+          assertEquals((float)2.0 * c, vector.vector[i], 0);
+          assertFalse(vector.isNull[i]);
+          c++;
+        }
+      }
+      assertEquals(nElements, c);
+    } finally {
+      reader.close();
+    }
+  }
+
+  @Test
+  public void testBooleanRead() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(IOConstants.COLUMNS,"boolean_field");
+    conf.set(IOConstants.COLUMNS_TYPES, "boolean");
+    conf.setBoolean(ColumnProjectionUtils.READ_ALL_COLUMNS, false);
+    conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0");
+    VectorizedParquetRecordReader reader =
+      createParquetReader("message test { required boolean boolean_field;}", conf);
+    VectorizedRowBatch previous = reader.createValue();
+    try {
+      long c = 0;
+      while (reader.next(NullWritable.get(), previous)) {
+        LongColumnVector vector = (LongColumnVector) previous.cols[0];
+        assertTrue(vector.noNulls);
+        for (int i = 0; i < vector.vector.length; i++) {
+          if(c == nElements){
+            break;
+          }
+          int e = (c % 5 == 0) ? 1 : 0;
+          assertEquals(e, vector.vector[i]);
+          assertFalse(vector.isNull[i]);
+          c++;
+        }
+      }
+      assertEquals(nElements, c);
+    } finally {
+      reader.close();
+    }
+  }
+
+  @Test
+  public void testBinaryReadDictionaryEncoding() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(IOConstants.COLUMNS,"binary_field");
+    conf.set(IOConstants.COLUMNS_TYPES, "string");
+    conf.setBoolean(ColumnProjectionUtils.READ_ALL_COLUMNS, false);
+    conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0");
+    VectorizedParquetRecordReader reader =
+      createParquetReader("message test { required binary binary_field;}", conf);
+    VectorizedRowBatch previous = reader.createValue();
+    int c = 0;
+    try {
+      while (reader.next(NullWritable.get(), previous)) {
+        BytesColumnVector vector = (BytesColumnVector) previous.cols[0];
+        boolean noNull = true;
+        for (int i = 0; i < vector.vector.length; i++) {
+          if(c == nElements){
+            break;
+          }
+          if (c % 13 == 1) {
+            assertTrue(vector.isNull[i]);
+          } else {
+            assertFalse(vector.isNull[i]);
+            int binaryLen = c % 10;
+            String expected = new String(new char[binaryLen]).replace("\0", "x");
+            String actual = new String(ArrayUtils
+              .subarray(vector.vector[i], vector.start[i], vector.start[i] + vector.length[i]));
+            assertEquals("Failed at " + c, expected, actual);
+            noNull = false;
+          }
+          c++;
+        }
+        assertEquals("No Null check failed at " + c, noNull, vector.noNulls);
+        assertFalse(vector.isRepeating);
+      }
+      assertEquals(nElements, c);
+    } finally {
+      reader.close();
+    }
+  }
+
+  @Test
+  public void testBinaryRead() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(IOConstants.COLUMNS,"binary_field_non_repeating");
+    conf.set(IOConstants.COLUMNS_TYPES, "string");
+    conf.setBoolean(ColumnProjectionUtils.READ_ALL_COLUMNS, false);
+    conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, "0");
+    VectorizedParquetRecordReader reader =
+      createParquetReader("message test { required binary binary_field_non_repeating;}", conf);
+    VectorizedRowBatch previous = reader.createValue();
+    int c = 0;
+    try {
+      while (reader.next(NullWritable.get(), previous)) {
+        BytesColumnVector vector = (BytesColumnVector) previous.cols[0];
+        boolean noNull = true;
+        for (int i = 0; i < vector.vector.length; i++) {
+          if(c == nElements){
+            break;
+          }
+          String actual;
+          assertEquals("Null assert failed at " + c, isNulls[c], vector.isNull[i]);
+          if (!vector.isNull[i]) {
+            actual = new String(ArrayUtils
+              .subarray(vector.vector[i], vector.start[i], vector.start[i] + vector.length[i]));
+            assertEquals("failed at " + c, uniqueStrs[c], actual);
+          }else{
+            noNull = false;
+          }
+          c++;
+        }
+        assertEquals("No Null check failed at " + c, noNull, vector.noNulls);
+        assertFalse(vector.isRepeating);
+      }
+      assertEquals("It doesn't exit at expected position", nElements, c);
+    } finally {
+      reader.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/test/queries/clientpositive/parquet_types_non_dictionary_encoding_vectorization.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_types_non_dictionary_encoding_vectorization.q b/ql/src/test/queries/clientpositive/parquet_types_non_dictionary_encoding_vectorization.q
new file mode 100644
index 0000000..7de444f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/parquet_types_non_dictionary_encoding_vectorization.q
@@ -0,0 +1,94 @@
+set hive.mapred.mode=nonstrict;
+DROP TABLE parquet_types_staging;
+DROP TABLE parquet_types;
+
+set hive.vectorized.execution.enabled=true;
+set hive.vectorized.execution.reduce.enabled=true;
+set hive.vectorized.use.row.serde.deserialize=true;
+set hive.vectorized.use.vector.serde.deserialize=true;
+set hive.vectorized.execution.reduce.groupby.enabled = true;
+
+CREATE TABLE parquet_types_staging (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary string,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':';
+
+CREATE TABLE parquet_types (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary binary,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/parquet_non_dictionary_types.txt' OVERWRITE INTO TABLE
+parquet_types_staging;
+
+SELECT * FROM parquet_types_staging;
+
+INSERT OVERWRITE TABLE parquet_types
+SELECT cint, ctinyint, csmallint, cfloat, cdouble, cstring1, t, cchar, cvarchar,
+unhex(cbinary), m1, l1, st1, d FROM parquet_types_staging;
+
+-- test types in group by
+
+EXPLAIN SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+;
+
+SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+;
+
+EXPLAIN SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat;
+SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat;
+
+EXPLAIN SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar;
+SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar;
+
+EXPLAIN SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar;
+SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar;
+
+EXPLAIN SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1;
+SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1;
+
+EXPLAIN SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary;
+SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/test/queries/clientpositive/parquet_types_vectorization.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/parquet_types_vectorization.q b/ql/src/test/queries/clientpositive/parquet_types_vectorization.q
new file mode 100644
index 0000000..bb0e5b2
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/parquet_types_vectorization.q
@@ -0,0 +1,96 @@
+set hive.mapred.mode=nonstrict;
+DROP TABLE parquet_types_staging;
+DROP TABLE parquet_types;
+
+set hive.vectorized.execution.enabled=true;
+set hive.vectorized.execution.reduce.enabled=true;
+set hive.vectorized.use.row.serde.deserialize=true;
+set hive.vectorized.use.vector.serde.deserialize=true;
+set hive.vectorized.execution.reduce.groupby.enabled = true;
+
+CREATE TABLE parquet_types_staging (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary string,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':';
+
+CREATE TABLE parquet_types (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary binary,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/parquet_types.txt' OVERWRITE INTO TABLE parquet_types_staging;
+
+SELECT * FROM parquet_types_staging;
+
+INSERT OVERWRITE TABLE parquet_types
+SELECT cint, ctinyint, csmallint, cfloat, cdouble, cstring1, t, cchar, cvarchar,
+unhex(cbinary), m1, l1, st1, d FROM parquet_types_staging;
+
+-- test types in group by
+
+EXPLAIN SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+;
+
+SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+;
+
+EXPLAIN SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat;
+SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat;
+
+EXPLAIN SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar;
+SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar;
+
+EXPLAIN SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar;
+SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar;
+
+EXPLAIN SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1;
+SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1;
+
+EXPLAIN SELECT t, count(*) FROM parquet_types GROUP BY t ORDER BY t;
+SELECT t, count(*) FROM parquet_types GROUP BY t ORDER BY t;
+
+EXPLAIN SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary;
+SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out b/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out
index 8345132..e42453d 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_parquet.q.out
@@ -150,7 +150,7 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: tinyint)
                         Statistics: Num rows: 12288 Data size: 73728 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,input:float>), _col5 (type: struct<count:bigint,sum:double,variance:double>)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
             Execution mode: llap

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/test/results/clientpositive/llap/vectorized_parquet_types.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_parquet_types.q.out b/ql/src/test/results/clientpositive/llap/vectorized_parquet_types.q.out
index b49d5dd..0524cb3 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_parquet_types.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_parquet_types.q.out
@@ -250,19 +250,19 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 3 vectorized, llap
-      File Output Operator [FS_10]
-        Select Operator [SEL_9] (rows=11 width=11)
+      File Output Operator [FS_12]
+        Select Operator [SEL_11] (rows=11 width=11)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
         <-Reducer 2 [SIMPLE_EDGE] llap
           SHUFFLE [RS_6]
             Group By Operator [GBY_4] (rows=11 width=11)
               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["max(VALUE._col0)","min(VALUE._col1)","count(VALUE._col2)","avg(VALUE._col3)","stddev_pop(VALUE._col4)","max(VALUE._col5)"],keys:KEY._col0
-            <-Map 1 [SIMPLE_EDGE] llap
+            <-Map 1 [SIMPLE_EDGE] vectorized, llap
               SHUFFLE [RS_3]
                 PartitionCols:_col0
-                Group By Operator [GBY_2] (rows=22 width=11)
+                Group By Operator [GBY_10] (rows=22 width=11)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["max(cint)","min(csmallint)","count(cstring1)","avg(cfloat)","stddev_pop(cdouble)","max(cdecimal)"],keys:ctinyint
-                  Select Operator [SEL_1] (rows=22 width=11)
+                  Select Operator [SEL_9] (rows=22 width=11)
                     Output:["ctinyint","cint","csmallint","cstring1","cfloat","cdouble","cdecimal"]
                     TableScan [TS_0] (rows=22 width=11)
                       default@parquet_types,parquet_types,Tbl:COMPLETE,Col:NONE,Output:["cint","ctinyint","csmallint","cfloat","cdouble","cstring1","cdecimal"]


[26/35] hive git commit: HIVE-15219. LLAP: Allow additional slider global parameters to be set while creating the LLAP package. (Siddharth Seth, reviewed by Gopal V)

Posted by se...@apache.org.
HIVE-15219. LLAP: Allow additional slider global parameters to be set while creating the LLAP package. (Siddharth Seth, reviewed by Gopal V)


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

Branch: refs/heads/hive-14535
Commit: 678e6cac091931314f9afabe97f32c63f9036e0f
Parents: bbcff5e
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Nov 18 15:24:39 2016 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Nov 18 15:24:39 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hive/llap/cli/LlapOptionsProcessor.java  |  6 ++++++
 llap-server/src/main/resources/package.py           | 16 ++++++++++++++++
 llap-server/src/main/resources/templates.py         |  3 ++-
 3 files changed, 24 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/678e6cac/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java
index 8b98aee..e680dfc 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapOptionsProcessor.java
@@ -52,6 +52,7 @@ public class LlapOptionsProcessor {
   public static final String OPTION_JAVA_HOME = "javaHome"; // forward via config.json
   public static final String OPTION_HIVECONF = "hiveconf"; // llap-daemon-site if relevant parameter
   public static final String OPTION_SLIDER_AM_CONTAINER_MB = "slider-am-container-mb"; // forward as arg
+  public static final String OPTION_SLIDER_APPCONFIG_GLOBAL = "slider-appconfig-global"; // forward as arg
   public static final String OPTION_LLAP_QUEUE = "queue"; // forward via config.json
   public static final String OPTION_IO_THREADS = "iothreads"; // llap-daemon-site
 
@@ -256,6 +257,11 @@ public class LlapOptionsProcessor {
         .withLongOpt(OPTION_SLIDER_AM_CONTAINER_MB)
         .withDescription("The size of the slider AppMaster container in MB").create('b'));
 
+    options.addOption(OptionBuilder.withValueSeparator().hasArgs(2).withArgName("property=value")
+        .withLongOpt(OPTION_SLIDER_APPCONFIG_GLOBAL)
+        .withDescription("Property (key=value) to be set in the global section of the Slider appConfig")
+        .create());
+
     options.addOption(OptionBuilder.hasArg().withArgName(OPTION_IO_THREADS)
         .withLongOpt(OPTION_IO_THREADS).withDescription("executor per instance").create('t'));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/678e6cac/llap-server/src/main/resources/package.py
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/package.py b/llap-server/src/main/resources/package.py
index 9d61058..380c6a8 100644
--- a/llap-server/src/main/resources/package.py
+++ b/llap-server/src/main/resources/package.py
@@ -3,6 +3,7 @@
 import sys,os,stat
 import argparse
 from json import loads as json_parse
+from json import dumps as json_print
 from os.path import exists, join, relpath
 from time import gmtime, strftime
 import shutil
@@ -57,6 +58,19 @@ def zipdir(path, zip, prefix="."):
 			src = join(root, file)
 			dst = src.replace(path, prefix)
 			zip.write(src, dst)
+
+def slider_appconfig_global_property(arg):
+	kv = arg.split("=")
+	if len(kv) != 2:
+		raise argparse.ArgumentTypeError("Value must be split into two parts separated by =")
+	return tuple(kv)
+
+def construct_slider_site_global_string(kvs):
+	if not kvs:
+		return ""
+	kvs = map(lambda a : a[0], kvs)
+	return ",\n" + ",\n".join(["    %s:%s" % (json_print(k), json_print(v)) for (k,v) in kvs])
+
 	
 def main(args):
 	version = os.getenv("HIVE_VERSION")
@@ -74,6 +88,7 @@ def main(args):
 	parser.add_argument("--logger", default="RFA")
 	parser.add_argument("--chaosmonkey", type=int, default=0)
 	parser.add_argument("--slider-am-container-mb", type=int, default=1024)
+	parser.add_argument("--slider-appconfig-global", nargs='*', type=slider_appconfig_global_property, action='append')
 	parser.add_argument("--slider-keytab-dir", default="")
 	parser.add_argument("--slider-keytab", default="")
 	parser.add_argument("--slider-principal", default="")
@@ -128,6 +143,7 @@ def main(args):
 		"monkey_percentage" : monkey_percentage,
 		"monkey_enabled" : args.chaosmonkey > 0,
 		"slider.am.container.mb" : args.slider_am_container_mb,
+		"slider_appconfig_global_append": construct_slider_site_global_string(args.slider_appconfig_global),
 		"slider_am_jvm_heapsize" : slider_am_jvm_heapsize,
 		"slider_keytab_dir" : slider_keytab_dir,
 		"slider_keytab" : slider_keytab,

http://git-wip-us.apache.org/repos/asf/hive/blob/678e6cac/llap-server/src/main/resources/templates.py
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/templates.py b/llap-server/src/main/resources/templates.py
index 5684f33..36162bb 100644
--- a/llap-server/src/main/resources/templates.py
+++ b/llap-server/src/main/resources/templates.py
@@ -84,7 +84,8 @@ appConfig = """
     "internal.chaos.monkey.probability.amlaunchfailure": "0",
     "internal.chaos.monkey.probability.containerfailure": "%(monkey_percentage)d",
     "internal.chaos.monkey.interval.seconds": "%(monkey_interval)d",
-    "internal.chaos.monkey.enabled": "%(monkey_enabled)s"
+    "internal.chaos.monkey.enabled": "%(monkey_enabled)s"%(slider_appconfig_global_append)s
+
   },
   "components": {
     "slider-appmaster": {


[13/35] hive git commit: HIVE-15231: query on view with CTE and alias fails with table not found error (Aihua Xu, reviewed by Pengcheng Xiong, Xuefu Zhang)

Posted by se...@apache.org.
HIVE-15231: query on view with CTE and alias fails with table not found error (Aihua Xu, reviewed by Pengcheng Xiong, Xuefu Zhang)


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

Branch: refs/heads/hive-14535
Commit: cea954f5c93bc8830c9358ad16d3c39aaf9c9de8
Parents: 936df7a
Author: Aihua Xu <ai...@apache.org>
Authored: Thu Nov 17 16:15:47 2016 -0500
Committer: Aihua Xu <ai...@apache.org>
Committed: Fri Nov 18 09:08:09 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  4 +-
 ql/src/test/queries/clientpositive/cteViews.q   |  8 ++++
 .../test/results/clientpositive/cteViews.q.out  | 46 ++++++++++++++++++++
 3 files changed, 56 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cea954f5/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 8f5542b..47feeaf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -825,9 +825,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     qb.getParseInfo().setSrcForAlias(alias, tableTree);
 
-    // if alias to CTE contains the alias, we do not do the translation because
+    // if alias to CTE contains the table name, we do not do the translation because
     // cte is actually a subquery.
-    if (!this.aliasToCTEs.containsKey(alias)) {
+    if (!this.aliasToCTEs.containsKey(tabIdName)) {
       unparseTranslator.addTableNameTranslation(tableTree, SessionState.get().getCurrentDatabase());
       if (aliasIndex != 0) {
         unparseTranslator.addIdentifierTranslation((ASTNode) tabref.getChild(aliasIndex));

http://git-wip-us.apache.org/repos/asf/hive/blob/cea954f5/ql/src/test/queries/clientpositive/cteViews.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cteViews.q b/ql/src/test/queries/clientpositive/cteViews.q
index c076841..4567b47 100644
--- a/ql/src/test/queries/clientpositive/cteViews.q
+++ b/ql/src/test/queries/clientpositive/cteViews.q
@@ -38,4 +38,12 @@ use bug;
 select * from default.v;
 use default;
 drop view v;
+
+create view v as with src1 as (select key from src order by key limit 5)
+select * from src1 a where a.key is not null;
+
+describe extended v;
+select * from v;
+drop view v;
+
 drop database bug;

http://git-wip-us.apache.org/repos/asf/hive/blob/cea954f5/ql/src/test/results/clientpositive/cteViews.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cteViews.q.out b/ql/src/test/results/clientpositive/cteViews.q.out
index 6291784..eb3cfc0 100644
--- a/ql/src/test/results/clientpositive/cteViews.q.out
+++ b/ql/src/test/results/clientpositive/cteViews.q.out
@@ -232,6 +232,52 @@ POSTHOOK: query: drop view v
 POSTHOOK: type: DROPVIEW
 POSTHOOK: Input: default@v
 POSTHOOK: Output: default@v
+PREHOOK: query: create view v as with src1 as (select key from src order by key limit 5)
+select * from src1 a where a.key is not null
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@v
+POSTHOOK: query: create view v as with src1 as (select key from src order by key limit 5)
+select * from src1 a where a.key is not null
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@v
+PREHOOK: query: describe extended v
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@v
+POSTHOOK: query: describe extended v
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@v
+key                 	string              	                    
+	 	 
+#### A masked pattern was here ####
+select * from src1 a where a.key is not null, viewExpandedText:with src1 as (select `src`.`key` from `default`.`src` order by key limit 5)	 	 
+select `a`.`key` from src1 a where `a`.`key` is not null, tableType:VIRTUAL_VIEW)		 
+PREHOOK: query: select * from v
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@v
+#### A masked pattern was here ####
+POSTHOOK: query: select * from v
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@v
+#### A masked pattern was here ####
+0
+0
+0
+10
+100
+PREHOOK: query: drop view v
+PREHOOK: type: DROPVIEW
+PREHOOK: Input: default@v
+PREHOOK: Output: default@v
+POSTHOOK: query: drop view v
+POSTHOOK: type: DROPVIEW
+POSTHOOK: Input: default@v
+POSTHOOK: Output: default@v
 PREHOOK: query: drop database bug
 PREHOOK: type: DROPDATABASE
 PREHOOK: Input: database:bug


[29/35] hive git commit: HIVE-15072: Schematool should recognize missing tables in metastore (Naveen Gangam via Chaoyu Tang)

Posted by se...@apache.org.
HIVE-15072: Schematool should recognize missing tables in metastore (Naveen Gangam via Chaoyu Tang)


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

Branch: refs/heads/hive-14535
Commit: 929ebbaa095a807bfdf3169e4f57dac7b98c2191
Parents: d2cb327
Author: Chaoyu Tang <ct...@cloudera.com>
Authored: Sun Nov 20 20:20:37 2016 -0500
Committer: Chaoyu Tang <ct...@cloudera.com>
Committed: Sun Nov 20 20:20:37 2016 -0500

----------------------------------------------------------------------
 .../org/apache/hive/beeline/HiveSchemaTool.java | 110 +++++++++++++++++++
 .../org/apache/hive/beeline/TestSchemaTool.java |  40 ++++++-
 2 files changed, 149 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/929ebbaa/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
index d82c224..3402470 100644
--- a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
+++ b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
@@ -49,12 +49,16 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 public class HiveSchemaTool {
   private String userName = null;
@@ -300,6 +304,7 @@ public class HiveSchemaTool {
   public void doValidate() throws HiveMetaException {
     System.out.print("Starting metastore validation");
     validateSequences();
+    validateSchemaTables();
 
     System.out.print("Done with metastore validation");
   }
@@ -365,6 +370,111 @@ public class HiveSchemaTool {
     }
   }
 
+  boolean validateSchemaTables() throws HiveMetaException {
+    ResultSet rs              = null;
+    DatabaseMetaData metadata = null;
+    List<String> dbTables     = new ArrayList<String>();
+    List<String> schemaTables = new ArrayList<String>();
+    List<String> subScripts   = new ArrayList<String>();
+    Connection hmsConn        = getConnectionToMetastore(false);
+    String version            = getMetaStoreSchemaVersion(hmsConn);
+    hmsConn                   = getConnectionToMetastore(false);
+
+    System.out.println("Validating tables in the schema for version " + version);
+    try {
+      metadata       = hmsConn.getMetaData();
+      String[] types = {"TABLE"};
+      rs             = metadata.getTables(null, null, "%", types);
+      String table   = null;
+
+      while (rs.next()) {
+        table = rs.getString("TABLE_NAME");
+        dbTables.add(table.toLowerCase());
+        LOG.debug("Found table " + table + " in HMS dbstore");
+      }
+    } catch (SQLException e) {
+      throw new HiveMetaException(e);
+    } finally {
+      if (rs != null) {
+        try {
+          rs.close();
+        } catch (SQLException e) {
+          throw new HiveMetaException("Failed to close resultset", e);
+        }
+      }
+
+      if (hmsConn != null) {
+        try {
+          hmsConn.close();
+        } catch (SQLException e) {
+          throw new HiveMetaException("Failed to close metastore connection", e);
+        }
+      }
+    }
+
+    // parse the schema file to determine the tables that are expected to exist
+    // we are using oracle schema because it is simpler to parse, no quotes or backticks etc
+    String baseDir    = new File(metaStoreSchemaInfo.getMetaStoreScriptDir()).getParent();
+    String schemaFile = baseDir + "/oracle/hive-schema-" + version + ".oracle.sql";
+
+    try {
+      LOG.info("Parsing schema script " + schemaFile);
+      subScripts.addAll(findCreateTable(schemaFile, schemaTables));
+      while (subScripts.size() > 0) {
+        schemaFile = baseDir + "/oracle/" + subScripts.remove(0);
+        LOG.info("Parsing subscript " + schemaFile);
+        subScripts.addAll(findCreateTable(schemaFile, schemaTables));
+      }
+    } catch (Exception e) {
+      return false;
+    }
+
+    System.out.println("Expected (from schema definition) " + schemaTables.size() +
+        " tables, Found (from HMS metastore) " + dbTables.size() + " tables");
+
+    // now diff the lists
+    schemaTables.removeAll(dbTables);
+    if (schemaTables.size() > 0) {
+      System.out.println(schemaTables.size() + " tables [ " + Arrays.toString(schemaTables.toArray())
+          + " ] are missing from the database schema.");
+      return false;
+    } else {
+      System.out.println("Schema table validation successful");
+      return true;
+    }
+  }
+
+  private List<String> findCreateTable(String path, List<String> tableList) {
+    Matcher matcher                       = null;
+    String line                           = null;
+    List<String> subs                     = new ArrayList<String>();
+    final String NESTED_SCRIPT_IDENTIFIER = "@";
+    Pattern regexp                        = Pattern.compile("(CREATE TABLE(IF NOT EXISTS)*) (\\S+).*");
+
+    try (
+      BufferedReader reader = new BufferedReader(new FileReader(path));
+    ){
+      while ((line = reader.readLine()) != null) {
+        if (line.startsWith(NESTED_SCRIPT_IDENTIFIER)) {
+          int endIndex = (line.indexOf(";") > -1 ) ? line.indexOf(";") : line.length();
+          // remove the trailing SEMI-COLON if any
+          subs.add(line.substring(NESTED_SCRIPT_IDENTIFIER.length(), endIndex));
+          continue;
+        }
+        matcher = regexp.matcher(line);
+        if (matcher.find()) {
+          String table = matcher.group(3);
+          tableList.add(table.toLowerCase());
+          LOG.debug("Found table " + table + " in the schema");
+        }
+      }
+    } catch (IOException ex){
+      ex.printStackTrace();
+    }
+
+    return subs;
+  }
+
   /**
    *  Run pre-upgrade scripts corresponding to a given upgrade script,
    *  if any exist. The errors from pre-upgrade are ignored.

http://git-wip-us.apache.org/repos/asf/hive/blob/929ebbaa/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
index 8aa4173..2209c83 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
@@ -104,6 +104,44 @@ public class TestSchemaTool extends TestCase {
   }
 
   /**
+   * Test to validate that all tables exist in the HMS metastore.
+   * @throws Exception
+   */
+  public void testValidateSchemaTables() throws Exception {
+    schemaTool.doInit("2.0.0");
+
+    boolean isValid = (boolean)schemaTool.validateSchemaTables();
+    assertTrue(isValid);
+
+    // upgrade to 2.2.0 schema and re-validate
+    schemaTool.doUpgrade("2.2.0");
+    isValid = (boolean)schemaTool.validateSchemaTables();
+    assertTrue(isValid);
+
+    // Simulate a missing table scenario by renaming a couple of tables
+    String[] scripts = new String[] {
+        "RENAME TABLE SEQUENCE_TABLE to SEQUENCE_TABLE_RENAMED",
+        "RENAME TABLE NUCLEUS_TABLES to NUCLEUS_TABLES_RENAMED"
+    };
+
+    File scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = schemaTool.validateSchemaTables();
+    assertFalse(isValid);
+
+    // Restored the renamed tables
+    scripts = new String[] {
+        "RENAME TABLE SEQUENCE_TABLE_RENAMED to SEQUENCE_TABLE",
+        "RENAME TABLE NUCLEUS_TABLES_RENAMED to NUCLEUS_TABLES"
+    };
+
+    scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = schemaTool.validateSchemaTables();
+    assertTrue(isValid);
+   }
+
+  /**
    * Test dryrun of schema initialization
    * @throws Exception
    */
@@ -520,4 +558,4 @@ public class TestSchemaTool extends TestCase {
     out.close();
     return preUpgradeScript;
   }
-}
\ No newline at end of file
+}


[18/35] hive git commit: HIVE-14089 : complex type support in LLAP IO is broken (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
index ebbdf8d..d5f5f9d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
@@ -17,18 +17,26 @@
  */
 package org.apache.hadoop.hive.ql.io.orc.encoded;
 
+import org.apache.orc.impl.RunLengthByteReader;
+import org.apache.orc.impl.StreamName;
+
 import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.hive.common.io.encoded.EncodedColumnBatch;
 import org.apache.hadoop.hive.common.io.encoded.EncodedColumnBatch.ColumnStreamData;
 import org.apache.orc.CompressionCodec;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.TypeDescription.Category;
 import org.apache.orc.impl.PositionProvider;
 import org.apache.orc.impl.SettableUncompressedStream;
 import org.apache.orc.impl.TreeReaderFactory;
 import org.apache.orc.OrcProto;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class EncodedTreeReaderFactory extends TreeReaderFactory {
+  private static final Logger LOG = LoggerFactory.getLogger(EncodedTreeReaderFactory.class);
   /**
    * We choose to use a toy programming language, so we cannot use multiple inheritance.
    * If we could, we could have this inherit TreeReader to contain the common impl, and then
@@ -36,7 +44,7 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
    * Instead, we have a settable interface that the caller will cast to and call setBuffers.
    */
   public interface SettableTreeReader {
-    void setBuffers(ColumnStreamData[] streamBuffers, boolean sameStripe) throws IOException;
+    void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe) throws IOException;
   }
 
   public static class TimestampStreamReader extends TimestampTreeReader
@@ -84,8 +92,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -198,6 +207,12 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
+    public void seek(PositionProvider[] index) throws IOException {
+      // This string reader should simply redirect to its own seek (what other types already do).
+      this.seek(index[columnId]);
+    }
+
+    @Override
     public void seek(PositionProvider index) throws IOException {
       if (present != null) {
         if (_isFileCompressed) {
@@ -211,7 +226,7 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
         // data stream could be empty stream or already reached end of stream before present stream.
         // This can happen if all values in stream are nulls or last row group values are all null.
-        if (_dataStream.available() > 0) {
+        if (_dataStream != null && _dataStream.available() > 0) {
           if (_isFileCompressed) {
             index.getNext();
           }
@@ -222,14 +237,14 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
 
         // data stream could be empty stream or already reached end of stream before present stream.
         // This can happen if all values in stream are nulls or last row group values are all null.
-        if (_dataStream.available() > 0) {
+        if (_dataStream != null && _dataStream.available() > 0) {
           if (_isFileCompressed) {
             index.getNext();
           }
           ((StringDirectTreeReader) reader).getStream().seek(index);
         }
 
-        if (_lengthStream.available() > 0) {
+        if (_lengthStream != null && _lengthStream.available() > 0) {
           if (_isFileCompressed) {
             index.getNext();
           }
@@ -239,8 +254,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -371,8 +387,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -468,8 +485,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -571,8 +589,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -668,8 +687,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -758,8 +778,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -859,8 +880,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -979,8 +1001,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -1065,6 +1088,12 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
+    public void seek(PositionProvider[] index) throws IOException {
+      // This string reader should simply redirect to its own seek (what other types already do).
+      this.seek(index[columnId]);
+    }
+
+    @Override
     public void seek(PositionProvider index) throws IOException {
       if (present != null) {
         if (_isFileCompressed) {
@@ -1106,8 +1135,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -1233,6 +1263,12 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
+    public void seek(PositionProvider[] index) throws IOException {
+      // This string reader should simply redirect to its own seek (what other types already do).
+      this.seek(index[columnId]);
+    }
+
+    @Override
     public void seek(PositionProvider index) throws IOException {
       if (present != null) {
         if (_isFileCompressed) {
@@ -1274,8 +1310,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -1411,8 +1448,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -1511,8 +1549,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -1617,8 +1656,9 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    public void setBuffers(ColumnStreamData[] streamsData, boolean sameStripe)
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
         throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
       if (_presentStream != null) {
         _presentStream.setBuffers(StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
       }
@@ -1673,177 +1713,760 @@ public class EncodedTreeReaderFactory extends TreeReaderFactory {
     }
   }
 
-  public static TreeReader[] createEncodedTreeReader(int numCols,
-                                                     List<OrcProto.Type> types,
-                                                     List<OrcProto.ColumnEncoding> encodings,
-                                                     EncodedColumnBatch<OrcBatchKey> batch,
-                                                     CompressionCodec codec, boolean skipCorrupt,
-                                                     String writerTimezone) throws IOException {
-    TreeReader[] treeReaders = new TreeReader[numCols];
-    for (int i = 0; i < numCols; i++) {
-      int columnIndex = batch.getColumnIxs()[i];
-      ColumnStreamData[] streamBuffers = batch.getColumnData()[i];
-      OrcProto.Type columnType = types.get(columnIndex);
-
-      // EncodedColumnBatch is already decompressed, we don't really need to pass codec.
-      // But we need to know if the original data is compressed or not. This is used to skip
-      // positions in row index properly. If the file is originally compressed,
-      // then 1st position (compressed offset) in row index should be skipped to get
-      // uncompressed offset, else 1st position should not be skipped.
-      // TODO: there should be a better way to do this, code just needs to be modified
-      OrcProto.ColumnEncoding columnEncoding = encodings.get(columnIndex);
-
-      // stream buffers are arranged in enum order of stream kind
-      ColumnStreamData present = streamBuffers[OrcProto.Stream.Kind.PRESENT_VALUE],
-        data = streamBuffers[OrcProto.Stream.Kind.DATA_VALUE],
-        dictionary = streamBuffers[OrcProto.Stream.Kind.DICTIONARY_DATA_VALUE],
-        lengths = streamBuffers[OrcProto.Stream.Kind.LENGTH_VALUE],
-        secondary = streamBuffers[OrcProto.Stream.Kind.SECONDARY_VALUE];
-
-      switch (columnType.getKind()) {
-        case BINARY:
-          treeReaders[i] = BinaryStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setLengthStream(lengths)
-              .setCompressionCodec(codec)
-              .setColumnEncoding(columnEncoding)
-              .build();
-          break;
-        case BOOLEAN:
-          treeReaders[i] = BooleanStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setCompressionCodec(codec)
-              .build();
-          break;
-        case BYTE:
-          treeReaders[i] = ByteStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setCompressionCodec(codec)
-              .build();
-          break;
-        case SHORT:
-          treeReaders[i] = ShortStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setCompressionCodec(codec)
-              .setColumnEncoding(columnEncoding)
-              .build();
-          break;
-        case INT:
-          treeReaders[i] = IntStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setCompressionCodec(codec)
-              .setColumnEncoding(columnEncoding)
-              .build();
-          break;
-        case LONG:
-          treeReaders[i] = LongStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setCompressionCodec(codec)
-              .setColumnEncoding(columnEncoding)
-              .skipCorrupt(skipCorrupt)
-              .build();
-          break;
-        case FLOAT:
-          treeReaders[i] = FloatStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setCompressionCodec(codec)
-              .build();
-          break;
-        case DOUBLE:
-          treeReaders[i] = DoubleStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setCompressionCodec(codec)
-              .build();
-          break;
-        case CHAR:
-          treeReaders[i] = CharStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setMaxLength(columnType.getMaximumLength())
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setLengthStream(lengths)
-              .setDictionaryStream(dictionary)
-              .setCompressionCodec(codec)
-              .setColumnEncoding(columnEncoding)
-              .build();
-          break;
-        case VARCHAR:
-          treeReaders[i] = VarcharStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setMaxLength(columnType.getMaximumLength())
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setLengthStream(lengths)
-              .setDictionaryStream(dictionary)
-              .setCompressionCodec(codec)
-              .setColumnEncoding(columnEncoding)
-              .build();
-          break;
-        case STRING:
-          treeReaders[i] = StringStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setDataStream(data)
-              .setLengthStream(lengths)
-              .setDictionaryStream(dictionary)
-              .setCompressionCodec(codec)
-              .setColumnEncoding(columnEncoding)
-              .build();
-          break;
-        case DECIMAL:
-          treeReaders[i] = DecimalStreamReader.builder()
-              .setColumnIndex(columnIndex)
-              .setPrecision(columnType.getPrecision())
-              .setScale(columnType.getScale())
-              .setPresentStream(present)
-              .setValueStream(data)
-              .setScaleStream(secondary)
-              .setCompressionCodec(codec)
-              .setColumnEncoding(columnEncoding)
-              .build();
-          break;
-        case TIMESTAMP:
-          treeReaders[i] = TimestampStreamReader.builder()
+  public static StructTreeReader createRootTreeReader(TypeDescription schema,
+       List<OrcProto.ColumnEncoding> encodings, EncodedColumnBatch<OrcBatchKey> batch,
+       CompressionCodec codec, boolean skipCorrupt, String tz, int[] columnMapping)
+           throws IOException {
+    if (schema.getCategory() != Category.STRUCT) {
+      throw new AssertionError("Schema is not a struct: " + schema);
+    }
+    // Some child types may be excluded. Note that this can only happen at root level.
+    List<TypeDescription> children = schema.getChildren();
+    int childCount = children.size(), includedCount = 0;
+    for (int childIx = 0; childIx < childCount; ++childIx) {
+      if (!batch.hasData(children.get(childIx).getId())) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Column at " + childIx + " " + children.get(childIx).getId()
+              + ":" + children.get(childIx).toString() + " has no data");
+        }
+        continue;
+      }
+      ++includedCount;
+    }
+    TreeReader[] childReaders = new TreeReader[includedCount];
+    for (int schemaChildIx = 0, inclChildIx = -1; schemaChildIx < childCount; ++schemaChildIx) {
+      if (!batch.hasData(children.get(schemaChildIx).getId())) continue;
+      childReaders[++inclChildIx] = createEncodedTreeReader(
+          schema.getChildren().get(schemaChildIx), encodings, batch, codec, skipCorrupt, tz);
+      columnMapping[inclChildIx] = schemaChildIx;
+    }
+    return StructStreamReader.builder()
+        .setColumnIndex(0)
+        .setCompressionCodec(codec)
+        .setColumnEncoding(encodings.get(0))
+        .setChildReaders(childReaders)
+        .build();
+  }
+
+
+  private static TreeReader createEncodedTreeReader(TypeDescription schema,
+      List<OrcProto.ColumnEncoding> encodings, EncodedColumnBatch<OrcBatchKey> batch,
+      CompressionCodec codec, boolean skipCorrupt, String tz) throws IOException {
+      int columnIndex = schema.getId();
+    ColumnStreamData[] streamBuffers = batch.getColumnData(columnIndex);
+
+    // EncodedColumnBatch is already decompressed, we don't really need to pass codec.
+    // But we need to know if the original data is compressed or not. This is used to skip
+    // positions in row index properly. If the file is originally compressed,
+    // then 1st position (compressed offset) in row index should be skipped to get
+    // uncompressed offset, else 1st position should not be skipped.
+    // TODO: there should be a better way to do this, code just needs to be modified
+    OrcProto.ColumnEncoding columnEncoding = encodings.get(columnIndex);
+
+    // stream buffers are arranged in enum order of stream kind
+    ColumnStreamData present = streamBuffers[OrcProto.Stream.Kind.PRESENT_VALUE],
+      data = streamBuffers[OrcProto.Stream.Kind.DATA_VALUE],
+      dictionary = streamBuffers[OrcProto.Stream.Kind.DICTIONARY_DATA_VALUE],
+      lengths = streamBuffers[OrcProto.Stream.Kind.LENGTH_VALUE],
+      secondary = streamBuffers[OrcProto.Stream.Kind.SECONDARY_VALUE];
+
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("columnIndex: {} columnType: {} streamBuffers.length: {} columnEncoding: {}" +
+          " present: {} data: {} dictionary: {} lengths: {} secondary: {} tz: {}",
+          columnIndex, schema, streamBuffers.length, columnEncoding, present != null,
+          data != null, dictionary != null, lengths != null, secondary != null, tz);
+    }
+    switch (schema.getCategory()) {
+      case BINARY:
+      case BOOLEAN:
+      case BYTE:
+      case SHORT:
+      case INT:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case CHAR:
+      case VARCHAR:
+      case STRING:
+      case DECIMAL:
+      case TIMESTAMP:
+      case DATE:
+        return getPrimitiveTreeReaders(columnIndex, schema, codec, columnEncoding,
+            present, data, dictionary, lengths, secondary, skipCorrupt, tz);
+      case LIST:
+        TypeDescription elementType = schema.getChildren().get(0);
+        TreeReader elementReader = createEncodedTreeReader(
+            elementType, encodings, batch, codec, skipCorrupt, tz);
+        return ListStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setColumnEncoding(columnEncoding)
+            .setCompressionCodec(codec)
+            .setPresentStream(present)
+            .setLengthStream(lengths)
+            .setElementReader(elementReader)
+            .build();
+      case MAP:
+        TypeDescription keyType = schema.getChildren().get(0);
+        TypeDescription valueType = schema.getChildren().get(1);
+        TreeReader keyReader = createEncodedTreeReader(
+            keyType, encodings, batch, codec, skipCorrupt, tz);
+        TreeReader valueReader = createEncodedTreeReader(
+            valueType, encodings, batch, codec, skipCorrupt, tz);
+        return MapStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setColumnEncoding(columnEncoding)
+            .setCompressionCodec(codec)
+            .setPresentStream(present)
+            .setLengthStream(lengths)
+            .setKeyReader(keyReader)
+            .setValueReader(valueReader)
+            .build();
+      case STRUCT: {
+        int childCount = schema.getChildren().size();
+        TreeReader[] childReaders = new TreeReader[childCount];
+        for (int i = 0; i < childCount; i++) {
+          TypeDescription childType = schema.getChildren().get(i);
+          childReaders[i] = createEncodedTreeReader(
+              childType, encodings, batch, codec, skipCorrupt, tz);
+        }
+        return StructStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .setPresentStream(present)
+            .setChildReaders(childReaders)
+            .build();
+      }
+      case UNION: {
+        int childCount = schema.getChildren().size();
+        TreeReader[] childReaders = new TreeReader[childCount];
+        for (int i = 0; i < childCount; i++) {
+          TypeDescription childType = schema.getChildren().get(i);
+          childReaders[i] = createEncodedTreeReader(
+              childType, encodings, batch, codec, skipCorrupt, tz);
+        }
+        return UnionStreamReader.builder()
               .setColumnIndex(columnIndex)
-              .setPresentStream(present)
-              .setSecondsStream(data)
-              .setNanosStream(secondary)
               .setCompressionCodec(codec)
               .setColumnEncoding(columnEncoding)
-              .setWriterTimezone(writerTimezone)
-              .skipCorrupt(skipCorrupt)
-              .build();
-          break;
-        case DATE:
-          treeReaders[i] = DateStreamReader.builder()
-              .setColumnIndex(columnIndex)
               .setPresentStream(present)
               .setDataStream(data)
-              .setCompressionCodec(codec)
-              .setColumnEncoding(columnEncoding)
+              .setChildReaders(childReaders)
               .build();
-          break;
-        default:
-          throw new UnsupportedOperationException("Data type not supported yet! " + columnType);
+      }
+      default:
+        throw new UnsupportedOperationException("Data type not supported: " + schema);
+    }
+  }
+
+  private static TreeReader getPrimitiveTreeReaders(final int columnIndex,
+      TypeDescription columnType, CompressionCodec codec, OrcProto.ColumnEncoding columnEncoding,
+      ColumnStreamData present, ColumnStreamData data, ColumnStreamData dictionary,
+      ColumnStreamData lengths, ColumnStreamData secondary, boolean skipCorrupt, String tz)
+          throws IOException {
+    switch (columnType.getCategory()) {
+      case BINARY:
+        return BinaryStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setLengthStream(lengths)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .build();
+      case BOOLEAN:
+        return BooleanStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setCompressionCodec(codec)
+            .build();
+      case BYTE:
+        return ByteStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setCompressionCodec(codec)
+            .build();
+      case SHORT:
+        return ShortStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .build();
+      case INT:
+        return IntStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .build();
+      case LONG:
+        return LongStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .skipCorrupt(skipCorrupt)
+            .build();
+      case FLOAT:
+        return FloatStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setCompressionCodec(codec)
+            .build();
+      case DOUBLE:
+        return DoubleStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setCompressionCodec(codec)
+            .build();
+      case CHAR:
+        return CharStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setMaxLength(columnType.getMaxLength())
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setLengthStream(lengths)
+            .setDictionaryStream(dictionary)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .build();
+      case VARCHAR:
+        return VarcharStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setMaxLength(columnType.getMaxLength())
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setLengthStream(lengths)
+            .setDictionaryStream(dictionary)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .build();
+      case STRING:
+        return StringStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setLengthStream(lengths)
+            .setDictionaryStream(dictionary)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .build();
+      case DECIMAL:
+        return DecimalStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPrecision(columnType.getPrecision())
+            .setScale(columnType.getScale())
+            .setPresentStream(present)
+            .setValueStream(data)
+            .setScaleStream(secondary)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .build();
+      case TIMESTAMP:
+        return TimestampStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setSecondsStream(data)
+            .setNanosStream(secondary)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .setWriterTimezone(tz)
+            .skipCorrupt(skipCorrupt)
+            .build();
+      case DATE:
+        return DateStreamReader.builder()
+            .setColumnIndex(columnIndex)
+            .setPresentStream(present)
+            .setDataStream(data)
+            .setCompressionCodec(codec)
+            .setColumnEncoding(columnEncoding)
+            .build();
+    default:
+      throw new AssertionError("Not a primitive category: " + columnType.getCategory());
+    }
+  }
+
+  protected static class ListStreamReader extends ListTreeReader implements SettableTreeReader {
+    private boolean _isFileCompressed;
+    private SettableUncompressedStream _presentStream;
+    private SettableUncompressedStream _lengthStream;
+
+    public ListStreamReader(final int columnIndex,
+        final SettableUncompressedStream present, final SettableUncompressedStream lengthStream,
+        final OrcProto.ColumnEncoding columnEncoding, final boolean isFileCompressed,
+        final TreeReader elementReader) throws IOException {
+      super(columnIndex, present, lengthStream, columnEncoding, elementReader);
+      this._isFileCompressed = isFileCompressed;
+      this._presentStream = present;
+      this._lengthStream = lengthStream;
+    }
+
+    @Override
+    public void seek(PositionProvider[] index) throws IOException {
+      PositionProvider ownIndex = index[columnId];
+      if (present != null) {
+        if (_isFileCompressed) {
+          ownIndex.getNext();
+        }
+        present.seek(ownIndex);
+      }
+
+      // lengths stream could be empty stream or already reached end of stream before present stream.
+      // This can happen if all values in stream are nulls or last row group values are all null.
+      if (_lengthStream.available() > 0) {
+        if (_isFileCompressed) {
+          ownIndex.getNext();
+        }
+        lengths.seek(ownIndex);
+        elementReader.seek(index);
       }
     }
 
-    return treeReaders;
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      // Only our parent class can call this.
+      throw new IOException("Should never be called");
+    }
+
+    @Override
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
+        throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
+      if (_presentStream != null) {
+        _presentStream.setBuffers(
+            StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
+      }
+      if (_lengthStream != null) {
+        _lengthStream.setBuffers(
+            StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.LENGTH_VALUE]));
+      }
+
+      if (elementReader != null) {
+        ((SettableTreeReader) elementReader).setBuffers(batch, sameStripe);
+      }
+    }
+
+    public static class StreamReaderBuilder {
+      private int columnIndex;
+      private ColumnStreamData presentStream;
+      private ColumnStreamData lengthStream;
+      private CompressionCodec compressionCodec;
+      private OrcProto.ColumnEncoding columnEncoding;
+      private TreeReader elementReader;
+
+
+      public ListStreamReader.StreamReaderBuilder setColumnIndex(int columnIndex) {
+        this.columnIndex = columnIndex;
+        return this;
+      }
+
+      public ListStreamReader.StreamReaderBuilder setLengthStream(ColumnStreamData lengthStream) {
+        this.lengthStream = lengthStream;
+        return this;
+      }
+
+      public ListStreamReader.StreamReaderBuilder setPresentStream(ColumnStreamData presentStream) {
+        this.presentStream = presentStream;
+        return this;
+      }
+
+      public ListStreamReader.StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) {
+        this.columnEncoding = encoding;
+        return this;
+      }
+
+      public ListStreamReader.StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) {
+        this.compressionCodec = compressionCodec;
+        return this;
+      }
+
+      public ListStreamReader.StreamReaderBuilder setElementReader(TreeReader elementReader) {
+        this.elementReader = elementReader;
+        return this;
+      }
+
+      public ListStreamReader build() throws IOException {
+        SettableUncompressedStream present = StreamUtils
+            .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(),
+                presentStream);
+
+        SettableUncompressedStream length = StreamUtils
+            .createSettableUncompressedStream(OrcProto.Stream.Kind.LENGTH.name(),
+                lengthStream);
+
+        boolean isFileCompressed = compressionCodec != null;
+        return new ListStreamReader(columnIndex, present, length, columnEncoding, isFileCompressed,
+            elementReader);
+      }
+    }
+
+    public static ListStreamReader.StreamReaderBuilder builder() {
+      return new ListStreamReader.StreamReaderBuilder();
+    }
+  }
+
+  protected static class MapStreamReader extends MapTreeReader implements SettableTreeReader{
+    private boolean _isFileCompressed;
+    private SettableUncompressedStream _presentStream;
+    private SettableUncompressedStream _lengthStream;
+
+    public MapStreamReader(final int columnIndex,
+        final SettableUncompressedStream present, final SettableUncompressedStream lengthStream,
+        final OrcProto.ColumnEncoding columnEncoding, final boolean isFileCompressed,
+        final TreeReader keyReader, final TreeReader valueReader) throws IOException {
+      super(columnIndex, present, lengthStream, columnEncoding, keyReader, valueReader);
+      this._isFileCompressed = isFileCompressed;
+      this._presentStream = present;
+      this._lengthStream = lengthStream;
+    }
+
+    @Override
+    public void seek(PositionProvider[] index) throws IOException {
+      // We are not calling super.seek since we handle the present stream differently.
+      PositionProvider ownIndex = index[columnId];
+      if (present != null) {
+        if (_isFileCompressed) {
+          ownIndex.getNext();
+        }
+        present.seek(ownIndex);
+      }
+
+      // lengths stream could be empty stream or already reached end of stream before present stream.
+      // This can happen if all values in stream are nulls or last row group values are all null.
+      if (_lengthStream.available() > 0) {
+        if (_isFileCompressed) {
+          ownIndex.getNext();
+        }
+        lengths.seek(ownIndex);
+        keyReader.seek(index);
+        valueReader.seek(index);
+      }
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      // Only our parent class can call this.
+      throw new IOException("Should never be called");
+    }
+
+    @Override
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
+        throws IOException {
+     ColumnStreamData[] streamsData = batch.getColumnData(columnId);
+     if (_presentStream != null) {
+        _presentStream.setBuffers(
+            StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
+      }
+      if (_lengthStream != null) {
+        _lengthStream.setBuffers(
+            StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.LENGTH_VALUE]));
+      }
+
+      if (keyReader != null) {
+        ((SettableTreeReader) keyReader).setBuffers(batch, sameStripe);
+      }
+
+      if (valueReader != null) {
+        ((SettableTreeReader) valueReader).setBuffers(batch, sameStripe);
+      }
+    }
+
+    public static class StreamReaderBuilder {
+      private int columnIndex;
+      private ColumnStreamData presentStream;
+      private ColumnStreamData lengthStream;
+      private CompressionCodec compressionCodec;
+      private OrcProto.ColumnEncoding columnEncoding;
+      private TreeReader keyReader;
+      private TreeReader valueReader;
+
+
+      public MapStreamReader.StreamReaderBuilder setColumnIndex(int columnIndex) {
+        this.columnIndex = columnIndex;
+        return this;
+      }
+
+      public MapStreamReader.StreamReaderBuilder setLengthStream(ColumnStreamData lengthStream) {
+        this.lengthStream = lengthStream;
+        return this;
+      }
+
+      public MapStreamReader.StreamReaderBuilder setPresentStream(ColumnStreamData presentStream) {
+        this.presentStream = presentStream;
+        return this;
+      }
+
+      public MapStreamReader.StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) {
+        this.columnEncoding = encoding;
+        return this;
+      }
+
+      public MapStreamReader.StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) {
+        this.compressionCodec = compressionCodec;
+        return this;
+      }
+
+      public MapStreamReader.StreamReaderBuilder setKeyReader(TreeReader keyReader) {
+        this.keyReader = keyReader;
+        return this;
+      }
+
+      public MapStreamReader.StreamReaderBuilder setValueReader(TreeReader valueReader) {
+        this.valueReader = valueReader;
+        return this;
+      }
+
+      public MapStreamReader build() throws IOException {
+        SettableUncompressedStream present = StreamUtils
+            .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(),
+                presentStream);
+
+        SettableUncompressedStream length = StreamUtils
+            .createSettableUncompressedStream(OrcProto.Stream.Kind.LENGTH.name(),
+                lengthStream);
+
+        boolean isFileCompressed = compressionCodec != null;
+        return new MapStreamReader(columnIndex, present, length, columnEncoding, isFileCompressed,
+            keyReader, valueReader);
+      }
+    }
+
+    public static MapStreamReader.StreamReaderBuilder builder() {
+      return new MapStreamReader.StreamReaderBuilder();
+    }
+  }
+
+  protected static class StructStreamReader extends StructTreeReader
+      implements SettableTreeReader {
+    private boolean _isFileCompressed;
+    private SettableUncompressedStream _presentStream;
+
+    public StructStreamReader(final int columnIndex,
+        final SettableUncompressedStream present,
+        final OrcProto.ColumnEncoding columnEncoding, final boolean isFileCompressed,
+        final TreeReader[] childReaders) throws IOException {
+      super(columnIndex, present, columnEncoding, childReaders);
+      this._isFileCompressed = isFileCompressed;
+      this._presentStream = present;
+    }
+
+    @Override
+    public void seek(PositionProvider[] index) throws IOException {
+      PositionProvider ownIndex = index[columnId];
+      if (present != null) {
+        if (_isFileCompressed) {
+          ownIndex.getNext();
+        }
+        present.seek(ownIndex);
+      }
+      if (fields != null) {
+        for (TreeReader child : fields) {
+          child.seek(index);
+        }
+      }
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      // Only our parent class can call this.
+      throw new IOException("Should never be called");
+    }
+
+
+    @Override
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
+        throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
+      if (_presentStream != null) {
+        _presentStream.setBuffers(
+            StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
+      }
+      if (fields != null) {
+        for (TreeReader child : fields) {
+          ((SettableTreeReader) child).setBuffers(batch, sameStripe);
+        }
+      }
+    }
+
+    public static class StreamReaderBuilder {
+      private int columnIndex;
+      private ColumnStreamData presentStream;
+      private CompressionCodec compressionCodec;
+      private OrcProto.ColumnEncoding columnEncoding;
+      private TreeReader[] childReaders;
+
+
+      public StructStreamReader.StreamReaderBuilder setColumnIndex(int columnIndex) {
+        this.columnIndex = columnIndex;
+        return this;
+      }
+
+      public StructStreamReader.StreamReaderBuilder setPresentStream(ColumnStreamData presentStream) {
+        this.presentStream = presentStream;
+        return this;
+      }
+
+      public StructStreamReader.StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) {
+        this.columnEncoding = encoding;
+        return this;
+      }
+
+      public StructStreamReader.StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) {
+        this.compressionCodec = compressionCodec;
+        return this;
+      }
+
+      public StructStreamReader.StreamReaderBuilder setChildReaders(TreeReader[] childReaders) {
+        this.childReaders = childReaders;
+        return this;
+      }
+
+      public StructStreamReader build() throws IOException {
+        SettableUncompressedStream present = StreamUtils
+            .createSettableUncompressedStream(OrcProto.Stream.Kind.PRESENT.name(),
+                presentStream);
+
+        boolean isFileCompressed = compressionCodec != null;
+        return new StructStreamReader(columnIndex, present, columnEncoding, isFileCompressed,
+            childReaders);
+      }
+    }
+
+    public static StructStreamReader.StreamReaderBuilder builder() {
+      return new StructStreamReader.StreamReaderBuilder();
+    }
+  }
+
+  protected static class UnionStreamReader extends UnionTreeReader implements SettableTreeReader {
+    private boolean _isFileCompressed;
+    private SettableUncompressedStream _presentStream;
+    private SettableUncompressedStream _dataStream;
+
+    public UnionStreamReader(final int columnIndex,
+        final SettableUncompressedStream present, final SettableUncompressedStream dataStream,
+        final OrcProto.ColumnEncoding columnEncoding, final boolean isFileCompressed,
+        final TreeReader[] childReaders) throws IOException {
+      super(columnIndex, present, columnEncoding, childReaders);
+      this._isFileCompressed = isFileCompressed;
+      this._presentStream = present;
+      this._dataStream = dataStream;
+      // Note: other parent readers init everything in ctor, but union does it in startStripe.
+      this.tags = new RunLengthByteReader(dataStream);
+    }
+
+    @Override
+    public void seek(PositionProvider[] index) throws IOException {
+      PositionProvider ownIndex = index[columnId];
+      if (present != null) {
+        if (_isFileCompressed) {
+          ownIndex.getNext();
+        }
+        present.seek(ownIndex);
+      }
+
+      // lengths stream could be empty stream or already reached end of stream before present stream.
+      // This can happen if all values in stream are nulls or last row group values are all null.
+      if (_dataStream.available() > 0) {
+        if (_isFileCompressed) {
+          ownIndex.getNext();
+        }
+        tags.seek(ownIndex);
+        if (fields != null) {
+          for (TreeReader child : fields) {
+            child.seek(index);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      // Only our parent class can call this.
+      throw new IOException("Should never be called");
+    }
+
+    @Override
+    public void setBuffers(EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe)
+        throws IOException {
+      ColumnStreamData[] streamsData = batch.getColumnData(columnId);
+      if (_presentStream != null) {
+        _presentStream.setBuffers(
+            StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.PRESENT_VALUE]));
+      }
+      if (_dataStream != null) {
+        _dataStream.setBuffers(
+            StreamUtils.createDiskRangeInfo(streamsData[OrcProto.Stream.Kind.DATA_VALUE]));
+      }
+      if (fields != null) {
+        for (TreeReader child : fields) {
+          ((SettableTreeReader) child).setBuffers(batch, sameStripe);
+        }
+      }
+    }
+
+    public static class StreamReaderBuilder {
+      private int columnIndex;
+      private ColumnStreamData presentStream;
+      private ColumnStreamData dataStream;
+      private CompressionCodec compressionCodec;
+      private OrcProto.ColumnEncoding columnEncoding;
+      private TreeReader[] childReaders;
+
+
+      public UnionStreamReader.StreamReaderBuilder setColumnIndex(int columnIndex) {
+        this.columnIndex = columnIndex;
+        return this;
+      }
+
+      public UnionStreamReader.StreamReaderBuilder setDataStream(ColumnStreamData dataStream) {
+        this.dataStream = dataStream;
+        return this;
+      }
+
+      public UnionStreamReader.StreamReaderBuilder setPresentStream(ColumnStreamData presentStream) {
+        this.presentStream = presentStream;
+        return this;
+      }
+
+      public UnionStreamReader.StreamReaderBuilder setColumnEncoding(OrcProto.ColumnEncoding encoding) {
+        this.columnEncoding = encoding;
+        return this;
+      }
+
+      public UnionStreamReader.StreamReaderBuilder setCompressionCodec(CompressionCodec compressionCodec) {
+        this.compressionCodec = compressionCodec;
+        return this;
+      }
+
+      public UnionStreamReader.StreamReaderBuilder setChildReaders(TreeReader[] childReaders) {
+        this.childReaders = childReaders;
+        return this;
+      }
+
+      public UnionStreamReader build() throws IOException {
+        SettableUncompressedStream present = StreamUtils.createSettableUncompressedStream(
+            OrcProto.Stream.Kind.PRESENT.name(), presentStream);
+
+        SettableUncompressedStream data = StreamUtils.createSettableUncompressedStream(
+            OrcProto.Stream.Kind.DATA.name(), dataStream);
+
+        boolean isFileCompressed = compressionCodec != null;
+        return new UnionStreamReader(columnIndex, present, data,
+            columnEncoding, isFileCompressed, childReaders);
+      }
+    }
+
+    public static UnionStreamReader.StreamReaderBuilder builder() {
+      return new UnionStreamReader.StreamReaderBuilder();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/Reader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/Reader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/Reader.java
index 4405232..1c5f0e6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/Reader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/Reader.java
@@ -46,7 +46,7 @@ public interface Reader extends org.apache.hadoop.hive.ql.io.orc.Reader {
     public static final int ALL_RGS = -1;
     /**
      * All the previous streams are data streams, this and the next ones are index streams.
-     * We assume the sort will stay the same for backward compat.
+     * We assume the order will stay the same for backward compat.
      */
     public static final int MAX_DATA_STREAMS = OrcProto.Stream.Kind.ROW_INDEX.getNumber();
     public void init(Object fileKey, int stripeIx, int rgIx, int columnCount) {
@@ -57,6 +57,10 @@ public interface Reader extends org.apache.hadoop.hive.ql.io.orc.Reader {
       }
       resetColumnArrays(columnCount);
     }
+
+    public void initOrcColumn(int colIx) {
+      super.initColumn(colIx, MAX_DATA_STREAMS);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
index 5cc3663..601324a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
@@ -278,30 +278,35 @@ public class MapWork extends BaseWork {
     }
 
     // check if the column types that are read are supported by LLAP IO
-    for (Map.Entry<String, Operator<? extends OperatorDesc>> entry : aliasToWork.entrySet()) {
-      if (hasLlap) {
-        final String alias = entry.getKey();
-        Operator<? extends OperatorDesc> op = entry.getValue();
-        PartitionDesc partitionDesc = aliasToPartnInfo.get(alias);
-        if (op instanceof TableScanOperator && partitionDesc != null &&
-            partitionDesc.getTableDesc() != null) {
-          final TableScanOperator tsOp = (TableScanOperator) op;
-          final List<String> readColumnNames = tsOp.getNeededColumns();
-          final Properties props = partitionDesc.getTableDesc().getProperties();
-          final List<TypeInfo> typeInfos = TypeInfoUtils.getTypeInfosFromTypeString(
-              props.getProperty(serdeConstants.LIST_COLUMN_TYPES));
-          final List<String> allColumnTypes = TypeInfoUtils.getTypeStringsFromTypeInfo(typeInfos);
-          final List<String> allColumnNames = Utilities.getColumnNames(props);
-          hasLlap = Utilities.checkLlapIOSupportedTypes(readColumnNames, allColumnNames,
-              allColumnTypes);
-        }
-      }
+    if (hasLlap) {
+      // TODO: no need for now hasLlap = checkVectorizerSupportedTypes();
     }
 
     llapIoDesc = deriveLlapIoDescString(
         isLlapOn, canWrapAny, hasPathToPartInfo, hasLlap, hasNonLlap, hasAcid);
   }
 
+  private boolean checkVectorizerSupportedTypes(boolean hasLlap) {
+    for (Map.Entry<String, Operator<? extends OperatorDesc>> entry : aliasToWork.entrySet()) {
+      final String alias = entry.getKey();
+      Operator<? extends OperatorDesc> op = entry.getValue();
+      PartitionDesc partitionDesc = aliasToPartnInfo.get(alias);
+      if (op instanceof TableScanOperator && partitionDesc != null &&
+          partitionDesc.getTableDesc() != null) {
+        final TableScanOperator tsOp = (TableScanOperator) op;
+        final List<String> readColumnNames = tsOp.getNeededColumns();
+        final Properties props = partitionDesc.getTableDesc().getProperties();
+        final List<TypeInfo> typeInfos = TypeInfoUtils.getTypeInfosFromTypeString(
+            props.getProperty(serdeConstants.LIST_COLUMN_TYPES));
+        final List<String> allColumnTypes = TypeInfoUtils.getTypeStringsFromTypeInfo(typeInfos);
+        final List<String> allColumnNames = Utilities.getColumnNames(props);
+        hasLlap = Utilities.checkVectorizerSupportedTypes(readColumnNames, allColumnNames,
+            allColumnTypes);
+      }
+    }
+    return hasLlap;
+  }
+
   private static String deriveLlapIoDescString(boolean isLlapOn, boolean canWrapAny,
       boolean hasPathToPartInfo, boolean hasLlap, boolean hasNonLlap, boolean hasAcid) {
     if (!isLlapOn) return null; // LLAP IO is off, don't output.

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/test/queries/clientpositive/vector_complex_all.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_complex_all.q b/ql/src/test/queries/clientpositive/vector_complex_all.q
index 91a7368..b71ac62 100644
--- a/ql/src/test/queries/clientpositive/vector_complex_all.q
+++ b/ql/src/test/queries/clientpositive/vector_complex_all.q
@@ -1,9 +1,11 @@
 set hive.compute.query.using.stats=false;
-set hive.compute.query.using.stats=false;
+set hive.strict.checks.cartesian.product=false;
 set hive.cli.print.header=true;
 set hive.explain.user=false;
 set hive.fetch.task.conversion=none;
 SET hive.vectorized.execution.enabled=true;
+set hive.llap.io.enabled=false;
+set hive.mapred.mode=nonstrict;
 
 CREATE TABLE orc_create_staging (
   str STRING,
@@ -21,25 +23,45 @@ CREATE TABLE orc_create_complex (
   str STRING,
   mp  MAP<STRING,STRING>,
   lst ARRAY<STRING>,
-  strct STRUCT<A:STRING,B:STRING>
-) STORED AS ORC;
+  strct STRUCT<A:STRING,B:STRING>,
+  val string
+) STORED AS ORC tblproperties("orc.row.index.stride"="1000", "orc.stripe.size"="1000", "orc.compress.size"="10000");
 
-INSERT OVERWRITE TABLE orc_create_complex SELECT * FROM orc_create_staging;
+INSERT OVERWRITE TABLE orc_create_complex
+SELECT orc_create_staging.*, '0' FROM orc_create_staging;
 
--- Since complex types are not supported, this query should not vectorize.
-EXPLAIN
-SELECT * FROM orc_create_complex;
+set hive.llap.io.enabled=true;
 
 SELECT * FROM orc_create_complex;
 
--- However, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT COUNT(*) FROM orc_create_complex;
+SELECT str FROM orc_create_complex;
+
+SELECT strct, mp, lst FROM orc_create_complex;
+
+SELECT lst, str FROM orc_create_complex;
+
+SELECT mp, str FROM orc_create_complex;
+
+SELECT strct, str FROM orc_create_complex;
+
+SELECT strct.B, str FROM orc_create_complex;
+
+set hive.llap.io.enabled=false;
+
+INSERT INTO TABLE orc_create_complex
+SELECT orc_create_staging.*, src1.key FROM orc_create_staging cross join src src1 cross join orc_create_staging spam1 cross join orc_create_staging spam2;
+
+select count(*) from orc_create_complex;
+
+set hive.llap.io.enabled=true;
+
+SELECT distinct lst, strct FROM orc_create_complex;
+
+SELECT str, count(val)  FROM orc_create_complex GROUP BY str;
+
+SELECT strct.B, count(val) FROM orc_create_complex GROUP BY strct.B;
+
+SELECT strct, mp, lst, str, count(val) FROM orc_create_complex GROUP BY strct, mp, lst, str;
 
-SELECT COUNT(*) FROM orc_create_complex;
 
--- Also, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT str FROM orc_create_complex ORDER BY str;
 
-SELECT str FROM orc_create_complex ORDER BY str;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/test/results/clientpositive/llap/vector_complex_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_complex_all.q.out b/ql/src/test/results/clientpositive/llap/vector_complex_all.q.out
index 08d49bc..f16bb16 100644
--- a/ql/src/test/results/clientpositive/llap/vector_complex_all.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_complex_all.q.out
@@ -34,8 +34,9 @@ PREHOOK: query: CREATE TABLE orc_create_complex (
   str STRING,
   mp  MAP<STRING,STRING>,
   lst ARRAY<STRING>,
-  strct STRUCT<A:STRING,B:STRING>
-) STORED AS ORC
+  strct STRUCT<A:STRING,B:STRING>,
+  val string
+) STORED AS ORC tblproperties("orc.row.index.stride"="1000", "orc.stripe.size"="1000", "orc.compress.size"="10000")
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@orc_create_complex
@@ -43,16 +44,19 @@ POSTHOOK: query: CREATE TABLE orc_create_complex (
   str STRING,
   mp  MAP<STRING,STRING>,
   lst ARRAY<STRING>,
-  strct STRUCT<A:STRING,B:STRING>
-) STORED AS ORC
+  strct STRUCT<A:STRING,B:STRING>,
+  val string
+) STORED AS ORC tblproperties("orc.row.index.stride"="1000", "orc.stripe.size"="1000", "orc.compress.size"="10000")
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@orc_create_complex
-PREHOOK: query: INSERT OVERWRITE TABLE orc_create_complex SELECT * FROM orc_create_staging
+PREHOOK: query: INSERT OVERWRITE TABLE orc_create_complex
+SELECT orc_create_staging.*, '0' FROM orc_create_staging
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_staging
 PREHOOK: Output: default@orc_create_complex
-POSTHOOK: query: INSERT OVERWRITE TABLE orc_create_complex SELECT * FROM orc_create_staging
+POSTHOOK: query: INSERT OVERWRITE TABLE orc_create_complex
+SELECT orc_create_staging.*, '0' FROM orc_create_staging
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_staging
 POSTHOOK: Output: default@orc_create_complex
@@ -60,199 +64,166 @@ POSTHOOK: Lineage: orc_create_complex.lst SIMPLE [(orc_create_staging)orc_create
 POSTHOOK: Lineage: orc_create_complex.mp SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:mp, type:map<string,string>, comment:null), ]
 POSTHOOK: Lineage: orc_create_complex.str SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:str, type:string, comment:null), ]
 POSTHOOK: Lineage: orc_create_complex.strct SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:strct, type:struct<A:string,B:string>, comment:null), ]
-orc_create_staging.str	orc_create_staging.mp	orc_create_staging.lst	orc_create_staging.strct
-PREHOOK: query: -- Since complex types are not supported, this query should not vectorize.
-EXPLAIN
-SELECT * FROM orc_create_complex
+POSTHOOK: Lineage: orc_create_complex.val SIMPLE []
+orc_create_staging.str	orc_create_staging.mp	orc_create_staging.lst	orc_create_staging.strct	c1
+PREHOOK: query: SELECT * FROM orc_create_complex
 PREHOOK: type: QUERY
-POSTHOOK: query: -- Since complex types are not supported, this query should not vectorize.
-EXPLAIN
-SELECT * FROM orc_create_complex
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM orc_create_complex
 POSTHOOK: type: QUERY
-Explain
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-
-STAGE PLANS:
-  Stage: Stage-1
-    Tez
+POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-      Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: orc_create_complex
-                  Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: str (type: string), mp (type: map<string,string>), lst (type: array<string>), strct (type: struct<a:string,b:string>)
-                    outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-                    File Output Operator
-                      compressed: false
-                      Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-                      table:
-                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-            Execution mode: llap
-            LLAP IO: no inputs
-
-  Stage: Stage-0
-    Fetch Operator
-      limit: -1
-      Processor Tree:
-        ListSink
-
-PREHOOK: query: SELECT * FROM orc_create_complex
+orc_create_complex.str	orc_create_complex.mp	orc_create_complex.lst	orc_create_complex.strct	orc_create_complex.val
+line1	{"key13":"value13","key11":"value11","key12":"value12"}	["a","b","c"]	{"a":"one","b":"two"}	0
+line2	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]	{"a":"three","b":"four"}	0
+line3	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]	{"a":"five","b":"six"}	0
+PREHOOK: query: SELECT str FROM orc_create_complex
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT * FROM orc_create_complex
+POSTHOOK: query: SELECT str FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+str
+line1
+line2
+line3
+PREHOOK: query: SELECT strct, mp, lst FROM orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT strct, mp, lst FROM orc_create_complex
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-orc_create_complex.str	orc_create_complex.mp	orc_create_complex.lst	orc_create_complex.strct
-line1	{"key13":"value13","key11":"value11","key12":"value12"}	["a","b","c"]	{"a":"one","b":"two"}
-line2	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]	{"a":"three","b":"four"}
-line3	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]	{"a":"five","b":"six"}
-PREHOOK: query: -- However, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT COUNT(*) FROM orc_create_complex
+strct	mp	lst
+{"a":"one","b":"two"}	{"key13":"value13","key11":"value11","key12":"value12"}	["a","b","c"]
+{"a":"three","b":"four"}	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]
+{"a":"five","b":"six"}	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]
+PREHOOK: query: SELECT lst, str FROM orc_create_complex
 PREHOOK: type: QUERY
-POSTHOOK: query: -- However, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT COUNT(*) FROM orc_create_complex
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT lst, str FROM orc_create_complex
 POSTHOOK: type: QUERY
-Explain
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-
-STAGE PLANS:
-  Stage: Stage-1
-    Tez
+POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-      Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+lst	str
+["a","b","c"]	line1
+["d","e","f"]	line2
+["g","h","i"]	line3
+PREHOOK: query: SELECT mp, str FROM orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-      Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: orc_create_complex
-                  Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: count()
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        sort order: 
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-        Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: count(VALUE._col0)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
-  Stage: Stage-0
-    Fetch Operator
-      limit: -1
-      Processor Tree:
-        ListSink
-
-PREHOOK: query: SELECT COUNT(*) FROM orc_create_complex
+POSTHOOK: query: SELECT mp, str FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+mp	str
+{"key13":"value13","key11":"value11","key12":"value12"}	line1
+{"key21":"value21","key22":"value22","key23":"value23"}	line2
+{"key31":"value31","key32":"value32","key33":"value33"}	line3
+PREHOOK: query: SELECT strct, str FROM orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT strct, str FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+strct	str
+{"a":"one","b":"two"}	line1
+{"a":"three","b":"four"}	line2
+{"a":"five","b":"six"}	line3
+PREHOOK: query: SELECT strct.B, str FROM orc_create_complex
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT COUNT(*) FROM orc_create_complex
+POSTHOOK: query: SELECT strct.B, str FROM orc_create_complex
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+b	str
+two	line1
+four	line2
+six	line3
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_1, $hdt$_2, $hdt$_3, $hdt$_0]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: INSERT INTO TABLE orc_create_complex
+SELECT orc_create_staging.*, src1.key FROM orc_create_staging cross join src src1 cross join orc_create_staging spam1 cross join orc_create_staging spam2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_staging
+PREHOOK: Input: default@src
+PREHOOK: Output: default@orc_create_complex
+POSTHOOK: query: INSERT INTO TABLE orc_create_complex
+SELECT orc_create_staging.*, src1.key FROM orc_create_staging cross join src src1 cross join orc_create_staging spam1 cross join orc_create_staging spam2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_staging
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@orc_create_complex
+POSTHOOK: Lineage: orc_create_complex.lst SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:lst, type:array<string>, comment:null), ]
+POSTHOOK: Lineage: orc_create_complex.mp SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:mp, type:map<string,string>, comment:null), ]
+POSTHOOK: Lineage: orc_create_complex.str SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:str, type:string, comment:null), ]
+POSTHOOK: Lineage: orc_create_complex.strct SIMPLE [(orc_create_staging)orc_create_staging.FieldSchema(name:strct, type:struct<A:string,B:string>, comment:null), ]
+POSTHOOK: Lineage: orc_create_complex.val SIMPLE [(src)src1.FieldSchema(name:key, type:string, comment:default), ]
+orc_create_staging.str	orc_create_staging.mp	orc_create_staging.lst	orc_create_staging.strct	src1.key
+PREHOOK: query: select count(*) from orc_create_complex
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from orc_create_complex
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
 c0
-3
-PREHOOK: query: -- Also, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT str FROM orc_create_complex ORDER BY str
+13503
+PREHOOK: query: SELECT distinct lst, strct FROM orc_create_complex
 PREHOOK: type: QUERY
-POSTHOOK: query: -- Also, since this query is not referencing the complex fields, it should vectorize.
-EXPLAIN
-SELECT str FROM orc_create_complex ORDER BY str
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT distinct lst, strct FROM orc_create_complex
 POSTHOOK: type: QUERY
-Explain
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-
-STAGE PLANS:
-  Stage: Stage-1
-    Tez
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+lst	strct
+["a","b","c"]	{"a":"one","b":"two"}
+["d","e","f"]	{"a":"three","b":"four"}
+["g","h","i"]	{"a":"five","b":"six"}
+PREHOOK: query: SELECT str, count(val)  FROM orc_create_complex GROUP BY str
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-      Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+POSTHOOK: query: SELECT str, count(val)  FROM orc_create_complex GROUP BY str
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-      Vertices:
-        Map 1 
-            Map Operator Tree:
-                TableScan
-                  alias: orc_create_complex
-                  Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: str (type: string)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-        Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY.reducesinkkey0 (type: string)
-                outputColumnNames: _col0
-                Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 3 Data size: 3177 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
-  Stage: Stage-0
-    Fetch Operator
-      limit: -1
-      Processor Tree:
-        ListSink
-
-PREHOOK: query: SELECT str FROM orc_create_complex ORDER BY str
+str	c1
+line1	4501
+line2	4501
+line3	4501
+PREHOOK: query: SELECT strct.B, count(val) FROM orc_create_complex GROUP BY strct.B
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT str FROM orc_create_complex ORDER BY str
+POSTHOOK: query: SELECT strct.B, count(val) FROM orc_create_complex GROUP BY strct.B
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-str
-line1
-line2
-line3
+strct.b	_c1
+four	4501
+six	4501
+two	4501
+PREHOOK: query: SELECT strct, mp, lst, str, count(val) FROM orc_create_complex GROUP BY strct, mp, lst, str
+PREHOOK: type: QUERY
+PREHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT strct, mp, lst, str, count(val) FROM orc_create_complex GROUP BY strct, mp, lst, str
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@orc_create_complex
+#### A masked pattern was here ####
+strct	mp	lst	str	c4
+{"a":"one","b":"two"}	{"key11":"value11","key12":"value12","key13":"value13"}	["a","b","c"]	line1	4501
+{"a":"three","b":"four"}	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]	line2	4501
+{"a":"five","b":"six"}	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]	line3	4501

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out b/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
index 97d5642..133b8ef 100644
--- a/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
@@ -90,7 +90,7 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 190 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: map<int,string>)
             Execution mode: llap
-            LLAP IO: no inputs
+            LLAP IO: all inputs
 
   Stage: Stage-0
     Fetch Operator
@@ -211,7 +211,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: a (type: array<int>)
             Execution mode: llap
-            LLAP IO: no inputs
+            LLAP IO: all inputs
 
   Stage: Stage-0
     Fetch Operator


[10/35] hive git commit: HIVE-14815: Implement Parquet vectorization reader for Primitive types(Ferdinand Xu, review by Chao Sun) This closes #104

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/test/results/clientpositive/parquet_types_non_dictionary_encoding_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_types_non_dictionary_encoding_vectorization.q.out b/ql/src/test/results/clientpositive/parquet_types_non_dictionary_encoding_vectorization.q.out
new file mode 100644
index 0000000..a9f5e48
--- /dev/null
+++ b/ql/src/test/results/clientpositive/parquet_types_non_dictionary_encoding_vectorization.q.out
@@ -0,0 +1,2452 @@
+PREHOOK: query: DROP TABLE parquet_types_staging
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE parquet_types_staging
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: DROP TABLE parquet_types
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: DROP TABLE parquet_types
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE parquet_types_staging (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary string,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_types_staging
+POSTHOOK: query: CREATE TABLE parquet_types_staging (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary string,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_types_staging
+PREHOOK: query: CREATE TABLE parquet_types (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary binary,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) STORED AS PARQUET
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@parquet_types
+POSTHOOK: query: CREATE TABLE parquet_types (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string,
+  t timestamp,
+  cchar char(5),
+  cvarchar varchar(10),
+  cbinary binary,
+  m1 map<string, varchar(3)>,
+  l1 array<int>,
+  st1 struct<c1:int, c2:char(1)>,
+  d date
+) STORED AS PARQUET
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@parquet_types
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/parquet_non_dictionary_types.txt' OVERWRITE INTO TABLE
+parquet_types_staging
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@parquet_types_staging
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/parquet_non_dictionary_types.txt' OVERWRITE INTO TABLE
+parquet_types_staging
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@parquet_types_staging
+PREHOOK: query: SELECT * FROM parquet_types_staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types_staging
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM parquet_types_staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types_staging
+#### A masked pattern was here ####
+1000	-128	0	0.0	0.3		1940-01-01 01:01:01.111111111	     			{"":""}	[1000,1001]	{"c1":1000,"c2":"b"}	1940-01-01
+1001	-127	1	0.3	1.3	b	1941-02-02 01:01:01.111111111	b    	b	b	{"b":"b"}	[1001,1002]	{"c1":1001,"c2":"c"}	1941-02-02
+1002	-126	2	0.6	2.3	c	1942-03-03 01:01:01.111111111	c    	c	c	{"c":"c"}	[1002,1003]	{"c1":1002,"c2":"d"}	1942-03-03
+1003	-125	3	0.9	3.3	d	1943-04-04 01:01:01.111111111	d    	d	d	{"d":"d"}	[1003,1004]	{"c1":1003,"c2":"e"}	1943-04-04
+1004	-124	4	1.2	4.3	e	1944-05-05 01:01:01.111111111	e    	e	e	{"e":"e"}	[1004,1005]	{"c1":1004,"c2":"f"}	1944-05-05
+1005	-123	5	1.5	5.3	f	1945-06-06 01:01:01.111111111	f    	f	f	{"f":"f"}	[1005,1006]	{"c1":1005,"c2":"g"}	1945-06-06
+1006	-122	6	1.8	6.3	g	1946-07-07 01:01:01.111111111	g    	g	g	{"g":"g"}	[1006,1007]	{"c1":1006,"c2":"h"}	1946-07-07
+1007	-121	7	2.1	7.3	h	1947-08-08 01:01:01.111111111	h    	h	h	{"h":"h"}	[1007,1008]	{"c1":1007,"c2":"i"}	1947-08-08
+1008	-120	8	2.4	8.3	i	1948-09-09 01:01:01.111111111	i    	i	i	{"i":"i"}	[1008,1009]	{"c1":1008,"c2":"j"}	1948-09-09
+1009	-119	9	2.7	9.3	j	1949-10-10 01:01:01.111111111	j    	j	j	{"j":"j"}	[1009,1010]	{"c1":1009,"c2":"k"}	1949-10-10
+1010	-118	10	3.0	10.3	k	1950-11-11 01:01:01.111111111	k    	k	k	{"k":"k"}	[1010,1011]	{"c1":1010,"c2":"l"}	1950-11-11
+1011	-117	11	3.3	11.3	l	1951-12-12 01:01:01.111111111	l    	l	l	{"l":"l"}	[1011,1012]	{"c1":1011,"c2":"m"}	1951-12-12
+1012	-116	12	3.6	12.3	m	1952-01-13 01:01:01.111111111	m    	m	m	{"m":"m"}	[1012,1013]	{"c1":1012,"c2":"n"}	1952-01-13
+1013	-115	13	3.9	13.3	n	1953-02-14 01:01:01.111111111	n    	n	n	{"n":"n"}	[1013,1014]	{"c1":1013,"c2":"o"}	1953-02-14
+1014	-114	14	4.2	14.3	o	1954-03-15 01:01:01.111111111	o    	o	o	{"o":"o"}	[1014,1015]	{"c1":1014,"c2":"p"}	1954-03-15
+1015	-113	15	4.5	15.3	p	1955-04-16 01:01:01.111111111	p    	p	p	{"p":"p"}	[1015,1016]	{"c1":1015,"c2":"q"}	1955-04-16
+1016	-112	16	4.8	16.3	q	1956-05-17 01:01:01.111111111	q    	q	q	{"q":"q"}	[1016,1017]	{"c1":1016,"c2":"r"}	1956-05-17
+1017	-111	17	5.1	17.3	r	1957-06-18 01:01:01.111111111	r    	r	r	{"r":"r"}	[1017,1018]	{"c1":1017,"c2":"s"}	1957-06-18
+1018	-110	18	5.4	18.3	s	1958-07-19 01:01:01.111111111	s    	s	s	{"s":"s"}	[1018,1019]	{"c1":1018,"c2":"t"}	1958-07-19
+1019	-109	19	5.7	19.3	t	1959-08-20 01:01:01.111111111	t    	t	t	{"t":"t"}	[1019,1020]	{"c1":1019,"c2":"u"}	1959-08-20
+1020	-108	20	6.0	20.3	u	1960-09-21 01:01:01.111111111	u    	u	u	{"u":"u"}	[1020,1021]	{"c1":1020,"c2":"v"}	1960-09-21
+1021	-107	21	6.3	21.3	v	1961-10-22 01:01:01.111111111	v    	v	v	{"v":"v"}	[1021,1022]	{"c1":1021,"c2":"w"}	1961-10-22
+1022	-106	22	6.6	22.3	w	1962-11-23 01:01:01.111111111	w    	w	w	{"w":"w"}	[1022,1023]	{"c1":1022,"c2":"x"}	1962-11-23
+1023	-105	23	6.9	23.3	x	1963-12-24 01:01:01.111111111	x    	x	x	{"x":"x"}	[1023,1024]	{"c1":1023,"c2":"y"}	1963-12-24
+1024	-104	24	7.2	24.3	y	1964-01-25 01:01:01.111111111	y    	y	y	{"y":"y"}	[1024,1025]	{"c1":1024,"c2":"z"}	1964-01-25
+1025	-103	25	7.5	25.3	z	1965-02-26 01:01:01.111111111	z    	z	z	{"z":"z"}	[1025,1026]	{"c1":1025,"c2":"b"}	1965-02-26
+1026	-102	26	7.8	26.3	ba	1966-03-27 01:01:01.111111111	ba   	ba	ba	{"ba":"ba"}	[1026,1027]	{"c1":1026,"c2":"b"}	1966-03-27
+1027	-101	27	8.1	27.3	bb	1967-04-01 01:01:01.111111111	bb   	bb	bb	{"bb":"bb"}	[1027,1028]	{"c1":1027,"c2":"b"}	1967-04-01
+1028	-100	28	8.4	28.3	bc	1968-05-02 01:01:01.111111111	bc   	bc	bc	{"bc":"bc"}	[1028,1029]	{"c1":1028,"c2":"b"}	1968-05-02
+1029	-99	29	8.7	29.3	bd	1969-06-03 01:01:01.111111111	bd   	bd	bd	{"bd":"bd"}	[1029,1030]	{"c1":1029,"c2":"b"}	1969-06-03
+1030	-98	30	9.0	30.3	be	1970-07-04 01:01:01.111111111	be   	be	be	{"be":"be"}	[1030,1031]	{"c1":1030,"c2":"b"}	1970-07-04
+1031	-97	31	9.3	31.3	bf	1971-08-05 01:01:01.111111111	bf   	bf	bf	{"bf":"bf"}	[1031,1032]	{"c1":1031,"c2":"b"}	1971-08-05
+1032	-96	32	9.6	32.3	bg	1972-09-06 01:01:01.111111111	bg   	bg	bg	{"bg":"bg"}	[1032,1033]	{"c1":1032,"c2":"b"}	1972-09-06
+1033	-95	33	9.9	33.3	bh	1973-10-07 01:01:01.111111111	bh   	bh	bh	{"bh":"bh"}	[1033,1034]	{"c1":1033,"c2":"b"}	1973-10-07
+1034	-94	34	10.2	34.3	bi	1974-11-08 01:01:01.111111111	bi   	bi	bi	{"bi":"bi"}	[1034,1035]	{"c1":1034,"c2":"b"}	1974-11-08
+1035	-93	35	10.5	35.3	bj	1975-12-09 01:01:01.111111111	bj   	bj	bj	{"bj":"bj"}	[1035,1036]	{"c1":1035,"c2":"b"}	1975-12-09
+1036	-92	36	10.8	36.3	bk	1976-01-10 01:01:01.111111111	bk   	bk	bk	{"bk":"bk"}	[1036,1037]	{"c1":1036,"c2":"b"}	1976-01-10
+1037	-91	37	11.1	37.3	bl	1977-02-11 01:01:01.111111111	bl   	bl	bl	{"bl":"bl"}	[1037,1038]	{"c1":1037,"c2":"b"}	1977-02-11
+1038	-90	38	11.4	38.3	bm	1978-03-12 01:01:01.111111111	bm   	bm	bm	{"bm":"bm"}	[1038,1039]	{"c1":1038,"c2":"b"}	1978-03-12
+1039	-89	39	11.7	39.3	bn	1979-04-13 01:01:01.111111111	bn   	bn	bn	{"bn":"bn"}	[1039,1040]	{"c1":1039,"c2":"b"}	1979-04-13
+1040	-88	40	12.0	40.3	bo	1980-05-14 01:01:01.111111111	bo   	bo	bo	{"bo":"bo"}	[1040,1041]	{"c1":1040,"c2":"b"}	1980-05-14
+1041	-87	41	12.3	41.3	bp	1981-06-15 01:01:01.111111111	bp   	bp	bp	{"bp":"bp"}	[1041,1042]	{"c1":1041,"c2":"b"}	1981-06-15
+1042	-86	42	12.6	42.3	bq	1982-07-16 01:01:01.111111111	bq   	bq	bq	{"bq":"bq"}	[1042,1043]	{"c1":1042,"c2":"b"}	1982-07-16
+1043	-85	43	12.9	43.3	br	1983-08-17 01:01:01.111111111	br   	br	br	{"br":"br"}	[1043,1044]	{"c1":1043,"c2":"b"}	1983-08-17
+1044	-84	44	13.2	44.3	bs	1984-09-18 01:01:01.111111111	bs   	bs	bs	{"bs":"bs"}	[1044,1045]	{"c1":1044,"c2":"b"}	1984-09-18
+1045	-83	45	13.5	45.3	bt	1985-10-19 01:01:01.111111111	bt   	bt	bt	{"bt":"bt"}	[1045,1046]	{"c1":1045,"c2":"b"}	1985-10-19
+1046	-82	46	13.8	46.3	bu	1986-11-20 01:01:01.111111111	bu   	bu	bu	{"bu":"bu"}	[1046,1047]	{"c1":1046,"c2":"b"}	1986-11-20
+1047	-81	47	14.1	47.3	bv	1987-12-21 01:01:01.111111111	bv   	bv	bv	{"bv":"bv"}	[1047,1048]	{"c1":1047,"c2":"b"}	1987-12-21
+1048	-80	48	14.4	48.3	bw	1988-01-22 01:01:01.111111111	bw   	bw	bw	{"bw":"bw"}	[1048,1049]	{"c1":1048,"c2":"b"}	1988-01-22
+1049	-79	49	14.7	49.3	bx	1989-02-23 01:01:01.111111111	bx   	bx	bx	{"bx":"bx"}	[1049,1050]	{"c1":1049,"c2":"b"}	1989-02-23
+1050	-78	50	15.0	50.3	by	1990-03-24 01:01:01.111111111	by   	by	by	{"by":"by"}	[1050,1051]	{"c1":1050,"c2":"b"}	1990-03-24
+1051	-77	51	15.3	51.3	bz	1991-04-25 01:01:01.111111111	bz   	bz	bz	{"bz":"bz"}	[1051,1052]	{"c1":1051,"c2":"c"}	1991-04-25
+1052	-76	52	15.6	52.3	ca	1992-05-26 01:01:01.111111111	ca   	ca	ca	{"ca":"ca"}	[1052,1053]	{"c1":1052,"c2":"c"}	1992-05-26
+1053	-75	53	15.9	53.3	cb	1993-06-27 01:01:01.111111111	cb   	cb	cb	{"cb":"cb"}	[1053,1054]	{"c1":1053,"c2":"c"}	1993-06-27
+1054	-74	54	16.2	54.3	cc	1994-07-01 01:01:01.111111111	cc   	cc	cc	{"cc":"cc"}	[1054,1055]	{"c1":1054,"c2":"c"}	1994-07-01
+1055	-73	55	16.5	55.3	cd	1995-08-02 01:01:01.111111111	cd   	cd	cd	{"cd":"cd"}	[1055,1056]	{"c1":1055,"c2":"c"}	1995-08-02
+1056	-72	56	16.8	56.3	ce	1996-09-03 01:01:01.111111111	ce   	ce	ce	{"ce":"ce"}	[1056,1057]	{"c1":1056,"c2":"c"}	1996-09-03
+1057	-71	57	17.1	57.3	cf	1997-10-04 01:01:01.111111111	cf   	cf	cf	{"cf":"cf"}	[1057,1058]	{"c1":1057,"c2":"c"}	1997-10-04
+1058	-70	58	17.4	58.3	cg	1998-11-05 01:01:01.111111111	cg   	cg	cg	{"cg":"cg"}	[1058,1059]	{"c1":1058,"c2":"c"}	1998-11-05
+1059	-69	59	17.7	59.3	ch	1999-12-06 01:01:01.111111111	ch   	ch	ch	{"ch":"ch"}	[1059,1060]	{"c1":1059,"c2":"c"}	1999-12-06
+1060	-68	60	18.0	60.3	ci	2000-01-07 01:01:01.111111111	ci   	ci	ci	{"ci":"ci"}	[1060,1061]	{"c1":1060,"c2":"c"}	2000-01-07
+1061	-67	61	18.3	61.3	cj	2001-02-08 01:01:01.111111111	cj   	cj	cj	{"cj":"cj"}	[1061,1062]	{"c1":1061,"c2":"c"}	2001-02-08
+1062	-66	62	18.6	62.3	ck	2002-03-09 01:01:01.111111111	ck   	ck	ck	{"ck":"ck"}	[1062,1063]	{"c1":1062,"c2":"c"}	2002-03-09
+1063	-65	63	18.9	63.3	cl	2003-04-10 01:01:01.111111111	cl   	cl	cl	{"cl":"cl"}	[1063,1064]	{"c1":1063,"c2":"c"}	2003-04-10
+1064	-64	64	19.2	64.3	cm	2004-05-11 01:01:01.111111111	cm   	cm	cm	{"cm":"cm"}	[1064,1065]	{"c1":1064,"c2":"c"}	2004-05-11
+1065	-63	65	19.5	65.3	cn	2005-06-12 01:01:01.111111111	cn   	cn	cn	{"cn":"cn"}	[1065,1066]	{"c1":1065,"c2":"c"}	2005-06-12
+1066	-62	66	19.8	66.3	co	2006-07-13 01:01:01.111111111	co   	co	co	{"co":"co"}	[1066,1067]	{"c1":1066,"c2":"c"}	2006-07-13
+1067	-61	67	20.1	67.3	cp	2007-08-14 01:01:01.111111111	cp   	cp	cp	{"cp":"cp"}	[1067,1068]	{"c1":1067,"c2":"c"}	2007-08-14
+1068	-60	68	20.4	68.3	cq	2008-09-15 01:01:01.111111111	cq   	cq	cq	{"cq":"cq"}	[1068,1069]	{"c1":1068,"c2":"c"}	2008-09-15
+1069	-59	69	20.7	69.3	cr	2009-10-16 01:01:01.111111111	cr   	cr	cr	{"cr":"cr"}	[1069,1070]	{"c1":1069,"c2":"c"}	2009-10-16
+1070	-58	70	21.0	70.3	cs	2010-11-17 01:01:01.111111111	cs   	cs	cs	{"cs":"cs"}	[1070,1071]	{"c1":1070,"c2":"c"}	2010-11-17
+1071	-57	71	21.3	71.3	ct	2011-12-18 01:01:01.111111111	ct   	ct	ct	{"ct":"ct"}	[1071,1072]	{"c1":1071,"c2":"c"}	2011-12-18
+1072	-56	72	21.6	72.3	cu	2012-01-19 01:01:01.111111111	cu   	cu	cu	{"cu":"cu"}	[1072,1073]	{"c1":1072,"c2":"c"}	2012-01-19
+1073	-55	73	21.9	73.3	cv	2013-02-20 01:01:01.111111111	cv   	cv	cv	{"cv":"cv"}	[1073,1074]	{"c1":1073,"c2":"c"}	2013-02-20
+1074	-54	74	22.2	74.3	cw	2014-03-21 01:01:01.111111111	cw   	cw	cw	{"cw":"cw"}	[1074,1075]	{"c1":1074,"c2":"c"}	2014-03-21
+1075	-53	75	22.5	75.3	cx	2015-04-22 01:01:01.111111111	cx   	cx	cx	{"cx":"cx"}	[1075,1076]	{"c1":1075,"c2":"c"}	2015-04-22
+1076	-52	76	22.8	76.3	cy	2016-05-23 01:01:01.111111111	cy   	cy	cy	{"cy":"cy"}	[1076,1077]	{"c1":1076,"c2":"c"}	2016-05-23
+1077	-51	77	23.1	77.3	cz	2017-06-24 01:01:01.111111111	cz   	cz	cz	{"cz":"cz"}	[1077,1078]	{"c1":1077,"c2":"d"}	2017-06-24
+1078	-50	78	23.4	78.3	da	2018-07-25 01:01:01.111111111	da   	da	da	{"da":"da"}	[1078,1079]	{"c1":1078,"c2":"d"}	2018-07-25
+1079	-49	79	23.7	79.3	db	2019-08-26 01:01:01.111111111	db   	db	db	{"db":"db"}	[1079,1080]	{"c1":1079,"c2":"d"}	2019-08-26
+1080	-48	80	24.0	80.3	dc	2020-09-27 01:01:01.111111111	dc   	dc	dc	{"dc":"dc"}	[1080,1081]	{"c1":1080,"c2":"d"}	2020-09-27
+1081	-47	81	24.3	81.3	dd	2021-10-01 01:01:01.111111111	dd   	dd	dd	{"dd":"dd"}	[1081,1082]	{"c1":1081,"c2":"d"}	2021-10-01
+1082	-46	82	24.6	82.3	de	2022-11-02 01:01:01.111111111	de   	de	de	{"de":"de"}	[1082,1083]	{"c1":1082,"c2":"d"}	2022-11-02
+1083	-45	83	24.9	83.3	df	2023-12-03 01:01:01.111111111	df   	df	df	{"df":"df"}	[1083,1084]	{"c1":1083,"c2":"d"}	2023-12-03
+1084	-44	84	25.2	84.3	dg	2024-01-04 01:01:01.111111111	dg   	dg	dg	{"dg":"dg"}	[1084,1085]	{"c1":1084,"c2":"d"}	2024-01-04
+1085	-43	85	25.5	85.3	dh	2025-02-05 01:01:01.111111111	dh   	dh	dh	{"dh":"dh"}	[1085,1086]	{"c1":1085,"c2":"d"}	2025-02-05
+1086	-42	86	25.8	86.3	di	2026-03-06 01:01:01.111111111	di   	di	di	{"di":"di"}	[1086,1087]	{"c1":1086,"c2":"d"}	2026-03-06
+1087	-41	87	26.1	87.3	dj	2027-04-07 01:01:01.111111111	dj   	dj	dj	{"dj":"dj"}	[1087,1088]	{"c1":1087,"c2":"d"}	2027-04-07
+1088	-40	88	26.4	88.3	dk	2028-05-08 01:01:01.111111111	dk   	dk	dk	{"dk":"dk"}	[1088,1089]	{"c1":1088,"c2":"d"}	2028-05-08
+1089	-39	89	26.7	89.3	dl	2029-06-09 01:01:01.111111111	dl   	dl	dl	{"dl":"dl"}	[1089,1090]	{"c1":1089,"c2":"d"}	2029-06-09
+1090	-38	90	27.0	90.3	dm	2030-07-10 01:01:01.111111111	dm   	dm	dm	{"dm":"dm"}	[1090,1091]	{"c1":1090,"c2":"d"}	2030-07-10
+1091	-37	91	27.3	91.3	dn	2031-08-11 01:01:01.111111111	dn   	dn	dn	{"dn":"dn"}	[1091,1092]	{"c1":1091,"c2":"d"}	2031-08-11
+1092	-36	92	27.6	92.3	do	2032-09-12 01:01:01.111111111	do   	do	do	{"do":"do"}	[1092,1093]	{"c1":1092,"c2":"d"}	2032-09-12
+1093	-35	93	27.9	93.3	dp	2033-10-13 01:01:01.111111111	dp   	dp	dp	{"dp":"dp"}	[1093,1094]	{"c1":1093,"c2":"d"}	2033-10-13
+1094	-34	94	28.2	94.3	dq	2034-11-14 01:01:01.111111111	dq   	dq	dq	{"dq":"dq"}	[1094,1095]	{"c1":1094,"c2":"d"}	2034-11-14
+1095	-33	95	28.5	95.3	dr	2035-12-15 01:01:01.111111111	dr   	dr	dr	{"dr":"dr"}	[1095,1096]	{"c1":1095,"c2":"d"}	2035-12-15
+1096	-32	96	28.8	96.3	ds	2036-01-16 01:01:01.111111111	ds   	ds	ds	{"ds":"ds"}	[1096,1097]	{"c1":1096,"c2":"d"}	2036-01-16
+1097	-31	97	29.1	97.3	dt	2037-02-17 01:01:01.111111111	dt   	dt	dt	{"dt":"dt"}	[1097,1098]	{"c1":1097,"c2":"d"}	2037-02-17
+1098	-30	98	29.4	98.3	du	2038-03-18 01:01:01.111111111	du   	du	du	{"du":"du"}	[1098,1099]	{"c1":1098,"c2":"d"}	2038-03-18
+1099	-29	99	29.7	99.3	dv	2039-04-19 01:01:01.111111111	dv   	dv	dv	{"dv":"dv"}	[1099,1100]	{"c1":1099,"c2":"d"}	2039-04-19
+1100	-28	100	30.0	100.3	dw	2040-05-20 01:01:01.111111111	dw   	dw	dw	{"dw":"dw"}	[1100,1101]	{"c1":1100,"c2":"d"}	2040-05-20
+1101	-27	101	30.3	101.3	dx	2041-06-21 01:01:01.111111111	dx   	dx	dx	{"dx":"dx"}	[1101,1102]	{"c1":1101,"c2":"d"}	2041-06-21
+1102	-26	102	30.6	102.3	dy	2042-07-22 01:01:01.111111111	dy   	dy	dy	{"dy":"dy"}	[1102,1103]	{"c1":1102,"c2":"d"}	2042-07-22
+1103	-25	103	30.9	103.3	dz	2043-08-23 01:01:01.111111111	dz   	dz	dz	{"dz":"dz"}	[1103,1104]	{"c1":1103,"c2":"e"}	2043-08-23
+1104	-24	104	31.2	104.3	ea	2044-09-24 01:01:01.111111111	ea   	ea	ea	{"ea":"ea"}	[1104,1105]	{"c1":1104,"c2":"e"}	2044-09-24
+1105	-23	105	31.5	105.3	eb	2045-10-25 01:01:01.111111111	eb   	eb	eb	{"eb":"eb"}	[1105,1106]	{"c1":1105,"c2":"e"}	2045-10-25
+1106	-22	106	31.8	106.3	ec	2046-11-26 01:01:01.111111111	ec   	ec	ec	{"ec":"ec"}	[1106,1107]	{"c1":1106,"c2":"e"}	2046-11-26
+1107	-21	107	32.1	107.3	ed	2047-12-27 01:01:01.111111111	ed   	ed	ed	{"ed":"ed"}	[1107,1108]	{"c1":1107,"c2":"e"}	2047-12-27
+1108	-20	108	32.4	108.3	ee	2048-01-01 01:01:01.111111111	ee   	ee	ee	{"ee":"ee"}	[1108,1109]	{"c1":1108,"c2":"e"}	2048-01-01
+1109	-19	109	32.7	109.3	ef	2049-02-02 01:01:01.111111111	ef   	ef	ef	{"ef":"ef"}	[1109,1110]	{"c1":1109,"c2":"e"}	2049-02-02
+1110	-18	110	33.0	110.3	eg	2050-03-03 01:01:01.111111111	eg   	eg	eg	{"eg":"eg"}	[1110,1111]	{"c1":1110,"c2":"e"}	2050-03-03
+1111	-17	111	33.3	111.3	eh	2051-04-04 01:01:01.111111111	eh   	eh	eh	{"eh":"eh"}	[1111,1112]	{"c1":1111,"c2":"e"}	2051-04-04
+1112	-16	112	33.6	112.3	ei	2052-05-05 01:01:01.111111111	ei   	ei	ei	{"ei":"ei"}	[1112,1113]	{"c1":1112,"c2":"e"}	2052-05-05
+1113	-15	113	33.9	113.3	ej	2053-06-06 01:01:01.111111111	ej   	ej	ej	{"ej":"ej"}	[1113,1114]	{"c1":1113,"c2":"e"}	2053-06-06
+1114	-14	114	34.2	114.3	ek	2054-07-07 01:01:01.111111111	ek   	ek	ek	{"ek":"ek"}	[1114,1115]	{"c1":1114,"c2":"e"}	2054-07-07
+1115	-13	115	34.5	115.3	el	2055-08-08 01:01:01.111111111	el   	el	el	{"el":"el"}	[1115,1116]	{"c1":1115,"c2":"e"}	2055-08-08
+1116	-12	116	34.8	116.3	em	2056-09-09 01:01:01.111111111	em   	em	em	{"em":"em"}	[1116,1117]	{"c1":1116,"c2":"e"}	2056-09-09
+1117	-11	117	35.1	117.3	en	2057-10-10 01:01:01.111111111	en   	en	en	{"en":"en"}	[1117,1118]	{"c1":1117,"c2":"e"}	2057-10-10
+1118	-10	118	35.4	118.3	eo	2058-11-11 01:01:01.111111111	eo   	eo	eo	{"eo":"eo"}	[1118,1119]	{"c1":1118,"c2":"e"}	2058-11-11
+1119	-9	119	35.7	119.3	ep	2059-12-12 01:01:01.111111111	ep   	ep	ep	{"ep":"ep"}	[1119,1120]	{"c1":1119,"c2":"e"}	2059-12-12
+1120	-8	120	36.0	120.3	eq	2060-01-13 01:01:01.111111111	eq   	eq	eq	{"eq":"eq"}	[1120,1121]	{"c1":1120,"c2":"e"}	2060-01-13
+1121	-7	121	36.3	121.3	er	2061-02-14 01:01:01.111111111	er   	er	er	{"er":"er"}	[1121,1122]	{"c1":1121,"c2":"e"}	2061-02-14
+1122	-6	122	36.6	122.3	es	2062-03-15 01:01:01.111111111	es   	es	es	{"es":"es"}	[1122,1123]	{"c1":1122,"c2":"e"}	2062-03-15
+1123	-5	123	36.9	123.3	et	2063-04-16 01:01:01.111111111	et   	et	et	{"et":"et"}	[1123,1124]	{"c1":1123,"c2":"e"}	2063-04-16
+1124	-4	124	37.2	124.3	eu	2064-05-17 01:01:01.111111111	eu   	eu	eu	{"eu":"eu"}	[1124,1125]	{"c1":1124,"c2":"e"}	2064-05-17
+1125	-3	125	37.5	125.3	ev	2065-06-18 01:01:01.111111111	ev   	ev	ev	{"ev":"ev"}	[1125,1126]	{"c1":1125,"c2":"e"}	2065-06-18
+1126	-2	126	37.8	126.3	ew	2066-07-19 01:01:01.111111111	ew   	ew	ew	{"ew":"ew"}	[1126,1127]	{"c1":1126,"c2":"e"}	2066-07-19
+1127	-1	127	38.1	127.3	ex	2067-08-20 01:01:01.111111111	ex   	ex	ex	{"ex":"ex"}	[1127,1128]	{"c1":1127,"c2":"e"}	2067-08-20
+1128	0	128	38.4	128.3	ey	2068-09-21 01:01:01.111111111	ey   	ey	ey	{"ey":"ey"}	[1128,1129]	{"c1":1128,"c2":"e"}	2068-09-21
+1129	1	129	38.7	129.3	ez	2069-10-22 01:01:01.111111111	ez   	ez	ez	{"ez":"ez"}	[1129,1130]	{"c1":1129,"c2":"f"}	2069-10-22
+1130	2	130	39.0	130.3	fa	2070-11-23 01:01:01.111111111	fa   	fa	fa	{"fa":"fa"}	[1130,1131]	{"c1":1130,"c2":"f"}	2070-11-23
+1131	3	131	39.3	131.3	fb	2071-12-24 01:01:01.111111111	fb   	fb	fb	{"fb":"fb"}	[1131,1132]	{"c1":1131,"c2":"f"}	2071-12-24
+1132	4	132	39.6	132.3	fc	2072-01-25 01:01:01.111111111	fc   	fc	fc	{"fc":"fc"}	[1132,1133]	{"c1":1132,"c2":"f"}	2072-01-25
+1133	5	133	39.9	133.3	fd	2073-02-26 01:01:01.111111111	fd   	fd	fd	{"fd":"fd"}	[1133,1134]	{"c1":1133,"c2":"f"}	2073-02-26
+1134	6	134	40.2	134.3	fe	2074-03-27 01:01:01.111111111	fe   	fe	fe	{"fe":"fe"}	[1134,1135]	{"c1":1134,"c2":"f"}	2074-03-27
+1135	7	135	40.5	135.3	ff	2075-04-01 01:01:01.111111111	ff   	ff	ff	{"ff":"ff"}	[1135,1136]	{"c1":1135,"c2":"f"}	2075-04-01
+1136	8	136	40.8	136.3	fg	2076-05-02 01:01:01.111111111	fg   	fg	fg	{"fg":"fg"}	[1136,1137]	{"c1":1136,"c2":"f"}	2076-05-02
+1137	9	137	41.1	137.3	fh	2077-06-03 01:01:01.111111111	fh   	fh	fh	{"fh":"fh"}	[1137,1138]	{"c1":1137,"c2":"f"}	2077-06-03
+1138	10	138	41.4	138.3	fi	2078-07-04 01:01:01.111111111	fi   	fi	fi	{"fi":"fi"}	[1138,1139]	{"c1":1138,"c2":"f"}	2078-07-04
+1139	11	139	41.7	139.3	fj	2079-08-05 01:01:01.111111111	fj   	fj	fj	{"fj":"fj"}	[1139,1140]	{"c1":1139,"c2":"f"}	2079-08-05
+1140	12	140	42.0	140.3	fk	2080-09-06 01:01:01.111111111	fk   	fk	fk	{"fk":"fk"}	[1140,1141]	{"c1":1140,"c2":"f"}	2080-09-06
+1141	13	141	42.3	141.3	fl	2081-10-07 01:01:01.111111111	fl   	fl	fl	{"fl":"fl"}	[1141,1142]	{"c1":1141,"c2":"f"}	2081-10-07
+1142	14	142	42.6	142.3	fm	2082-11-08 01:01:01.111111111	fm   	fm	fm	{"fm":"fm"}	[1142,1143]	{"c1":1142,"c2":"f"}	2082-11-08
+1143	15	143	42.9	143.3	fn	2083-12-09 01:01:01.111111111	fn   	fn	fn	{"fn":"fn"}	[1143,1144]	{"c1":1143,"c2":"f"}	2083-12-09
+1144	16	144	43.2	144.3	fo	2084-01-10 01:01:01.111111111	fo   	fo	fo	{"fo":"fo"}	[1144,1145]	{"c1":1144,"c2":"f"}	2084-01-10
+1145	17	145	43.5	145.3	fp	2085-02-11 01:01:01.111111111	fp   	fp	fp	{"fp":"fp"}	[1145,1146]	{"c1":1145,"c2":"f"}	2085-02-11
+1146	18	146	43.8	146.3	fq	2086-03-12 01:01:01.111111111	fq   	fq	fq	{"fq":"fq"}	[1146,1147]	{"c1":1146,"c2":"f"}	2086-03-12
+1147	19	147	44.1	147.3	fr	2087-04-13 01:01:01.111111111	fr   	fr	fr	{"fr":"fr"}	[1147,1148]	{"c1":1147,"c2":"f"}	2087-04-13
+1148	20	148	44.4	148.3	fs	2088-05-14 01:01:01.111111111	fs   	fs	fs	{"fs":"fs"}	[1148,1149]	{"c1":1148,"c2":"f"}	2088-05-14
+1149	21	149	44.7	149.3	ft	2089-06-15 01:01:01.111111111	ft   	ft	ft	{"ft":"ft"}	[1149,1150]	{"c1":1149,"c2":"f"}	2089-06-15
+1150	22	150	45.0	150.3	fu	2090-07-16 01:01:01.111111111	fu   	fu	fu	{"fu":"fu"}	[1150,1151]	{"c1":1150,"c2":"f"}	2090-07-16
+1151	23	151	45.3	151.3	fv	2091-08-17 01:01:01.111111111	fv   	fv	fv	{"fv":"fv"}	[1151,1152]	{"c1":1151,"c2":"f"}	2091-08-17
+1152	24	152	45.6	152.3	fw	2092-09-18 01:01:01.111111111	fw   	fw	fw	{"fw":"fw"}	[1152,1153]	{"c1":1152,"c2":"f"}	2092-09-18
+1153	25	153	45.9	153.3	fx	2093-10-19 01:01:01.111111111	fx   	fx	fx	{"fx":"fx"}	[1153,1154]	{"c1":1153,"c2":"f"}	2093-10-19
+1154	26	154	46.2	154.3	fy	2094-11-20 01:01:01.111111111	fy   	fy	fy	{"fy":"fy"}	[1154,1155]	{"c1":1154,"c2":"f"}	2094-11-20
+1155	27	155	46.5	155.3	fz	2095-12-21 01:01:01.111111111	fz   	fz	fz	{"fz":"fz"}	[1155,1156]	{"c1":1155,"c2":"g"}	2095-12-21
+1156	28	156	46.8	156.3	ga	2096-01-22 01:01:01.111111111	ga   	ga	ga	{"ga":"ga"}	[1156,1157]	{"c1":1156,"c2":"g"}	2096-01-22
+1157	29	157	47.1	157.3	gb	2097-02-23 01:01:01.111111111	gb   	gb	gb	{"gb":"gb"}	[1157,1158]	{"c1":1157,"c2":"g"}	2097-02-23
+1158	30	158	47.4	158.3	gc	2098-03-24 01:01:01.111111111	gc   	gc	gc	{"gc":"gc"}	[1158,1159]	{"c1":1158,"c2":"g"}	2098-03-24
+1159	31	159	47.7	159.3	gd	2099-04-25 01:01:01.111111111	gd   	gd	gd	{"gd":"gd"}	[1159,1160]	{"c1":1159,"c2":"g"}	2099-04-25
+1160	32	160	48.0	160.3	ge	2100-05-26 01:01:01.111111111	ge   	ge	ge	{"ge":"ge"}	[1160,1161]	{"c1":1160,"c2":"g"}	2100-05-26
+1161	33	161	48.3	161.3	gf	2101-06-27 01:01:01.111111111	gf   	gf	gf	{"gf":"gf"}	[1161,1162]	{"c1":1161,"c2":"g"}	2101-06-27
+1162	34	162	48.6	162.3	gg	2102-07-01 01:01:01.111111111	gg   	gg	gg	{"gg":"gg"}	[1162,1163]	{"c1":1162,"c2":"g"}	2102-07-01
+1163	35	163	48.9	163.3	gh	2103-08-02 01:01:01.111111111	gh   	gh	gh	{"gh":"gh"}	[1163,1164]	{"c1":1163,"c2":"g"}	2103-08-02
+1164	36	164	49.2	164.3	gi	2104-09-03 01:01:01.111111111	gi   	gi	gi	{"gi":"gi"}	[1164,1165]	{"c1":1164,"c2":"g"}	2104-09-03
+1165	37	165	49.5	165.3	gj	2105-10-04 01:01:01.111111111	gj   	gj	gj	{"gj":"gj"}	[1165,1166]	{"c1":1165,"c2":"g"}	2105-10-04
+1166	38	166	49.8	166.3	gk	2106-11-05 01:01:01.111111111	gk   	gk	gk	{"gk":"gk"}	[1166,1167]	{"c1":1166,"c2":"g"}	2106-11-05
+1167	39	167	50.1	167.3	gl	2107-12-06 01:01:01.111111111	gl   	gl	gl	{"gl":"gl"}	[1167,1168]	{"c1":1167,"c2":"g"}	2107-12-06
+1168	40	168	50.4	168.3	gm	2108-01-07 01:01:01.111111111	gm   	gm	gm	{"gm":"gm"}	[1168,1169]	{"c1":1168,"c2":"g"}	2108-01-07
+1169	41	169	50.7	169.3	gn	2109-02-08 01:01:01.111111111	gn   	gn	gn	{"gn":"gn"}	[1169,1170]	{"c1":1169,"c2":"g"}	2109-02-08
+1170	42	170	51.0	170.3	go	2110-03-09 01:01:01.111111111	go   	go	go	{"go":"go"}	[1170,1171]	{"c1":1170,"c2":"g"}	2110-03-09
+1171	43	171	51.3	171.3	gp	2111-04-10 01:01:01.111111111	gp   	gp	gp	{"gp":"gp"}	[1171,1172]	{"c1":1171,"c2":"g"}	2111-04-10
+1172	44	172	51.6	172.3	gq	2112-05-11 01:01:01.111111111	gq   	gq	gq	{"gq":"gq"}	[1172,1173]	{"c1":1172,"c2":"g"}	2112-05-11
+1173	45	173	51.9	173.3	gr	2113-06-12 01:01:01.111111111	gr   	gr	gr	{"gr":"gr"}	[1173,1174]	{"c1":1173,"c2":"g"}	2113-06-12
+1174	46	174	52.2	174.3	gs	2114-07-13 01:01:01.111111111	gs   	gs	gs	{"gs":"gs"}	[1174,1175]	{"c1":1174,"c2":"g"}	2114-07-13
+1175	47	175	52.5	175.3	gt	2115-08-14 01:01:01.111111111	gt   	gt	gt	{"gt":"gt"}	[1175,1176]	{"c1":1175,"c2":"g"}	2115-08-14
+1176	48	176	52.8	176.3	gu	2116-09-15 01:01:01.111111111	gu   	gu	gu	{"gu":"gu"}	[1176,1177]	{"c1":1176,"c2":"g"}	2116-09-15
+1177	49	177	53.1	177.3	gv	2117-10-16 01:01:01.111111111	gv   	gv	gv	{"gv":"gv"}	[1177,1178]	{"c1":1177,"c2":"g"}	2117-10-16
+1178	50	178	53.4	178.3	gw	2118-11-17 01:01:01.111111111	gw   	gw	gw	{"gw":"gw"}	[1178,1179]	{"c1":1178,"c2":"g"}	2118-11-17
+1179	51	179	53.7	179.3	gx	2119-12-18 01:01:01.111111111	gx   	gx	gx	{"gx":"gx"}	[1179,1180]	{"c1":1179,"c2":"g"}	2119-12-18
+1180	52	180	54.0	180.3	gy	2120-01-19 01:01:01.111111111	gy   	gy	gy	{"gy":"gy"}	[1180,1181]	{"c1":1180,"c2":"g"}	2120-01-19
+1181	53	181	54.3	181.3	gz	2121-02-20 01:01:01.111111111	gz   	gz	gz	{"gz":"gz"}	[1181,1182]	{"c1":1181,"c2":"h"}	2121-02-20
+1182	54	182	54.6	182.3	ha	2122-03-21 01:01:01.111111111	ha   	ha	ha	{"ha":"ha"}	[1182,1183]	{"c1":1182,"c2":"h"}	2122-03-21
+1183	55	183	54.9	183.3	hb	2123-04-22 01:01:01.111111111	hb   	hb	hb	{"hb":"hb"}	[1183,1184]	{"c1":1183,"c2":"h"}	2123-04-22
+1184	56	184	55.2	184.3	hc	2124-05-23 01:01:01.111111111	hc   	hc	hc	{"hc":"hc"}	[1184,1185]	{"c1":1184,"c2":"h"}	2124-05-23
+1185	57	185	55.5	185.3	hd	2125-06-24 01:01:01.111111111	hd   	hd	hd	{"hd":"hd"}	[1185,1186]	{"c1":1185,"c2":"h"}	2125-06-24
+1186	58	186	55.8	186.3	he	2126-07-25 01:01:01.111111111	he   	he	he	{"he":"he"}	[1186,1187]	{"c1":1186,"c2":"h"}	2126-07-25
+1187	59	187	56.1	187.3	hf	2127-08-26 01:01:01.111111111	hf   	hf	hf	{"hf":"hf"}	[1187,1188]	{"c1":1187,"c2":"h"}	2127-08-26
+1188	60	188	56.4	188.3	hg	2128-09-27 01:01:01.111111111	hg   	hg	hg	{"hg":"hg"}	[1188,1189]	{"c1":1188,"c2":"h"}	2128-09-27
+1189	61	189	56.7	189.3	hh	2129-10-01 01:01:01.111111111	hh   	hh	hh	{"hh":"hh"}	[1189,1190]	{"c1":1189,"c2":"h"}	2129-10-01
+1190	62	190	57.0	190.3	hi	2130-11-02 01:01:01.111111111	hi   	hi	hi	{"hi":"hi"}	[1190,1191]	{"c1":1190,"c2":"h"}	2130-11-02
+1191	63	191	57.3	191.3	hj	2131-12-03 01:01:01.111111111	hj   	hj	hj	{"hj":"hj"}	[1191,1192]	{"c1":1191,"c2":"h"}	2131-12-03
+1192	64	192	57.6	192.3	hk	2132-01-04 01:01:01.111111111	hk   	hk	hk	{"hk":"hk"}	[1192,1193]	{"c1":1192,"c2":"h"}	2132-01-04
+1193	65	193	57.9	193.3	hl	2133-02-05 01:01:01.111111111	hl   	hl	hl	{"hl":"hl"}	[1193,1194]	{"c1":1193,"c2":"h"}	2133-02-05
+1194	66	194	58.2	194.3	hm	2134-03-06 01:01:01.111111111	hm   	hm	hm	{"hm":"hm"}	[1194,1195]	{"c1":1194,"c2":"h"}	2134-03-06
+1195	67	195	58.5	195.3	hn	2135-04-07 01:01:01.111111111	hn   	hn	hn	{"hn":"hn"}	[1195,1196]	{"c1":1195,"c2":"h"}	2135-04-07
+1196	68	196	58.8	196.3	ho	2136-05-08 01:01:01.111111111	ho   	ho	ho	{"ho":"ho"}	[1196,1197]	{"c1":1196,"c2":"h"}	2136-05-08
+1197	69	197	59.1	197.3	hp	2137-06-09 01:01:01.111111111	hp   	hp	hp	{"hp":"hp"}	[1197,1198]	{"c1":1197,"c2":"h"}	2137-06-09
+1198	70	198	59.4	198.3	hq	2138-07-10 01:01:01.111111111	hq   	hq	hq	{"hq":"hq"}	[1198,1199]	{"c1":1198,"c2":"h"}	2138-07-10
+1199	71	199	59.7	199.3	hr	2139-08-11 01:01:01.111111111	hr   	hr	hr	{"hr":"hr"}	[1199,1200]	{"c1":1199,"c2":"h"}	2139-08-11
+1200	72	200	60.0	200.3	hs	2140-09-12 01:01:01.111111111	hs   	hs	hs	{"hs":"hs"}	[1200,1201]	{"c1":1200,"c2":"h"}	2140-09-12
+1201	73	201	60.3	201.3	ht	2141-10-13 01:01:01.111111111	ht   	ht	ht	{"ht":"ht"}	[1201,1202]	{"c1":1201,"c2":"h"}	2141-10-13
+1202	74	202	60.6	202.3	hu	2142-11-14 01:01:01.111111111	hu   	hu	hu	{"hu":"hu"}	[1202,1203]	{"c1":1202,"c2":"h"}	2142-11-14
+1203	75	203	60.9	203.3	hv	2143-12-15 01:01:01.111111111	hv   	hv	hv	{"hv":"hv"}	[1203,1204]	{"c1":1203,"c2":"h"}	2143-12-15
+1204	76	204	61.2	204.3	hw	2144-01-16 01:01:01.111111111	hw   	hw	hw	{"hw":"hw"}	[1204,1205]	{"c1":1204,"c2":"h"}	2144-01-16
+1205	77	205	61.5	205.3	hx	2145-02-17 01:01:01.111111111	hx   	hx	hx	{"hx":"hx"}	[1205,1206]	{"c1":1205,"c2":"h"}	2145-02-17
+1206	78	206	61.8	206.3	hy	2146-03-18 01:01:01.111111111	hy   	hy	hy	{"hy":"hy"}	[1206,1207]	{"c1":1206,"c2":"h"}	2146-03-18
+1207	79	207	62.1	207.3	hz	2147-04-19 01:01:01.111111111	hz   	hz	hz	{"hz":"hz"}	[1207,1208]	{"c1":1207,"c2":"i"}	2147-04-19
+1208	80	208	62.4	208.3	ia	2148-05-20 01:01:01.111111111	ia   	ia	ia	{"ia":"ia"}	[1208,1209]	{"c1":1208,"c2":"i"}	2148-05-20
+1209	81	209	62.7	209.3	ib	2149-06-21 01:01:01.111111111	ib   	ib	ib	{"ib":"ib"}	[1209,1210]	{"c1":1209,"c2":"i"}	2149-06-21
+1210	82	210	63.0	210.3	ic	2150-07-22 01:01:01.111111111	ic   	ic	ic	{"ic":"ic"}	[1210,1211]	{"c1":1210,"c2":"i"}	2150-07-22
+1211	83	211	63.3	211.3	id	2151-08-23 01:01:01.111111111	id   	id	id	{"id":"id"}	[1211,1212]	{"c1":1211,"c2":"i"}	2151-08-23
+1212	84	212	63.6	212.3	ie	2152-09-24 01:01:01.111111111	ie   	ie	ie	{"ie":"ie"}	[1212,1213]	{"c1":1212,"c2":"i"}	2152-09-24
+1213	85	213	63.9	213.3	if	2153-10-25 01:01:01.111111111	if   	if	if	{"if":"if"}	[1213,1214]	{"c1":1213,"c2":"i"}	2153-10-25
+1214	86	214	64.2	214.3	ig	2154-11-26 01:01:01.111111111	ig   	ig	ig	{"ig":"ig"}	[1214,1215]	{"c1":1214,"c2":"i"}	2154-11-26
+1215	87	215	64.5	215.3	ih	2155-12-27 01:01:01.111111111	ih   	ih	ih	{"ih":"ih"}	[1215,1216]	{"c1":1215,"c2":"i"}	2155-12-27
+1216	88	216	64.8	216.3	ii	2156-01-01 01:01:01.111111111	ii   	ii	ii	{"ii":"ii"}	[1216,1217]	{"c1":1216,"c2":"i"}	2156-01-01
+1217	89	217	65.1	217.3	ij	2157-02-02 01:01:01.111111111	ij   	ij	ij	{"ij":"ij"}	[1217,1218]	{"c1":1217,"c2":"i"}	2157-02-02
+1218	90	218	65.4	218.3	ik	2158-03-03 01:01:01.111111111	ik   	ik	ik	{"ik":"ik"}	[1218,1219]	{"c1":1218,"c2":"i"}	2158-03-03
+1219	91	219	65.7	219.3	il	2159-04-04 01:01:01.111111111	il   	il	il	{"il":"il"}	[1219,1220]	{"c1":1219,"c2":"i"}	2159-04-04
+1220	92	220	66.0	220.3	im	2160-05-05 01:01:01.111111111	im   	im	im	{"im":"im"}	[1220,1221]	{"c1":1220,"c2":"i"}	2160-05-05
+1221	93	221	66.3	221.3	in	2161-06-06 01:01:01.111111111	in   	in	in	{"in":"in"}	[1221,1222]	{"c1":1221,"c2":"i"}	2161-06-06
+1222	94	222	66.6	222.3	io	2162-07-07 01:01:01.111111111	io   	io	io	{"io":"io"}	[1222,1223]	{"c1":1222,"c2":"i"}	2162-07-07
+1223	95	223	66.9	223.3	ip	2163-08-08 01:01:01.111111111	ip   	ip	ip	{"ip":"ip"}	[1223,1224]	{"c1":1223,"c2":"i"}	2163-08-08
+1224	96	224	67.2	224.3	iq	2164-09-09 01:01:01.111111111	iq   	iq	iq	{"iq":"iq"}	[1224,1225]	{"c1":1224,"c2":"i"}	2164-09-09
+1225	97	225	67.5	225.3	ir	2165-10-10 01:01:01.111111111	ir   	ir	ir	{"ir":"ir"}	[1225,1226]	{"c1":1225,"c2":"i"}	2165-10-10
+1226	98	226	67.8	226.3	is	2166-11-11 01:01:01.111111111	is   	is	is	{"is":"is"}	[1226,1227]	{"c1":1226,"c2":"i"}	2166-11-11
+1227	99	227	68.1	227.3	it	2167-12-12 01:01:01.111111111	it   	it	it	{"it":"it"}	[1227,1228]	{"c1":1227,"c2":"i"}	2167-12-12
+1228	100	228	68.4	228.3	iu	2168-01-13 01:01:01.111111111	iu   	iu	iu	{"iu":"iu"}	[1228,1229]	{"c1":1228,"c2":"i"}	2168-01-13
+1229	101	229	68.7	229.3	iv	2169-02-14 01:01:01.111111111	iv   	iv	iv	{"iv":"iv"}	[1229,1230]	{"c1":1229,"c2":"i"}	2169-02-14
+1230	102	230	69.0	230.3	iw	2170-03-15 01:01:01.111111111	iw   	iw	iw	{"iw":"iw"}	[1230,1231]	{"c1":1230,"c2":"i"}	2170-03-15
+1231	103	231	69.3	231.3	ix	2171-04-16 01:01:01.111111111	ix   	ix	ix	{"ix":"ix"}	[1231,1232]	{"c1":1231,"c2":"i"}	2171-04-16
+1232	104	232	69.6	232.3	iy	2172-05-17 01:01:01.111111111	iy   	iy	iy	{"iy":"iy"}	[1232,1233]	{"c1":1232,"c2":"i"}	2172-05-17
+1233	105	233	69.9	233.3	iz	2173-06-18 01:01:01.111111111	iz   	iz	iz	{"iz":"iz"}	[1233,1234]	{"c1":1233,"c2":"j"}	2173-06-18
+1234	106	234	70.2	234.3	ja	2174-07-19 01:01:01.111111111	ja   	ja	ja	{"ja":"ja"}	[1234,1235]	{"c1":1234,"c2":"j"}	2174-07-19
+1235	107	235	70.5	235.3	jb	2175-08-20 01:01:01.111111111	jb   	jb	jb	{"jb":"jb"}	[1235,1236]	{"c1":1235,"c2":"j"}	2175-08-20
+1236	108	236	70.8	236.3	jc	2176-09-21 01:01:01.111111111	jc   	jc	jc	{"jc":"jc"}	[1236,1237]	{"c1":1236,"c2":"j"}	2176-09-21
+1237	109	237	71.1	237.3	jd	2177-10-22 01:01:01.111111111	jd   	jd	jd	{"jd":"jd"}	[1237,1238]	{"c1":1237,"c2":"j"}	2177-10-22
+1238	110	238	71.4	238.3	je	2178-11-23 01:01:01.111111111	je   	je	je	{"je":"je"}	[1238,1239]	{"c1":1238,"c2":"j"}	2178-11-23
+1239	111	239	71.7	239.3	jf	2179-12-24 01:01:01.111111111	jf   	jf	jf	{"jf":"jf"}	[1239,1240]	{"c1":1239,"c2":"j"}	2179-12-24
+1240	112	240	72.0	240.3	jg	2180-01-25 01:01:01.111111111	jg   	jg	jg	{"jg":"jg"}	[1240,1241]	{"c1":1240,"c2":"j"}	2180-01-25
+1241	113	241	72.3	241.3	jh	2181-02-26 01:01:01.111111111	jh   	jh	jh	{"jh":"jh"}	[1241,1242]	{"c1":1241,"c2":"j"}	2181-02-26
+1242	114	242	72.6	242.3	ji	2182-03-27 01:01:01.111111111	ji   	ji	ji	{"ji":"ji"}	[1242,1243]	{"c1":1242,"c2":"j"}	2182-03-27
+1243	115	243	72.9	243.3	jj	2183-04-01 01:01:01.111111111	jj   	jj	jj	{"jj":"jj"}	[1243,1244]	{"c1":1243,"c2":"j"}	2183-04-01
+1244	116	244	73.2	244.3	jk	2184-05-02 01:01:01.111111111	jk   	jk	jk	{"jk":"jk"}	[1244,1245]	{"c1":1244,"c2":"j"}	2184-05-02
+1245	117	245	73.5	245.3	jl	2185-06-03 01:01:01.111111111	jl   	jl	jl	{"jl":"jl"}	[1245,1246]	{"c1":1245,"c2":"j"}	2185-06-03
+1246	118	246	73.8	246.3	jm	2186-07-04 01:01:01.111111111	jm   	jm	jm	{"jm":"jm"}	[1246,1247]	{"c1":1246,"c2":"j"}	2186-07-04
+1247	119	247	74.1	247.3	jn	2187-08-05 01:01:01.111111111	jn   	jn	jn	{"jn":"jn"}	[1247,1248]	{"c1":1247,"c2":"j"}	2187-08-05
+1248	120	248	74.4	248.3	jo	2188-09-06 01:01:01.111111111	jo   	jo	jo	{"jo":"jo"}	[1248,1249]	{"c1":1248,"c2":"j"}	2188-09-06
+1249	121	249	74.7	249.3	jp	2189-10-07 01:01:01.111111111	jp   	jp	jp	{"jp":"jp"}	[1249,1250]	{"c1":1249,"c2":"j"}	2189-10-07
+1250	122	250	75.0	250.3	jq	2190-11-08 01:01:01.111111111	jq   	jq	jq	{"jq":"jq"}	[1250,1251]	{"c1":1250,"c2":"j"}	2190-11-08
+1251	123	251	75.3	251.3	jr	2191-12-09 01:01:01.111111111	jr   	jr	jr	{"jr":"jr"}	[1251,1252]	{"c1":1251,"c2":"j"}	2191-12-09
+1252	124	252	75.6	252.3	js	2192-01-10 01:01:01.111111111	js   	js	js	{"js":"js"}	[1252,1253]	{"c1":1252,"c2":"j"}	2192-01-10
+1253	125	253	75.9	253.3	jt	2193-02-11 01:01:01.111111111	jt   	jt	jt	{"jt":"jt"}	[1253,1254]	{"c1":1253,"c2":"j"}	2193-02-11
+1254	126	254	76.2	254.3	ju	2194-03-12 01:01:01.111111111	ju   	ju	ju	{"ju":"ju"}	[1254,1255]	{"c1":1254,"c2":"j"}	2194-03-12
+1255	127	255	76.5	255.3	jv	2195-04-13 01:01:01.111111111	jv   	jv	jv	{"jv":"jv"}	[1255,1256]	{"c1":1255,"c2":"j"}	2195-04-13
+1256	-128	256	76.8	256.3	jw	2196-05-14 01:01:01.111111111	jw   	jw	jw	{"jw":"jw"}	[1256,1257]	{"c1":1256,"c2":"j"}	2196-05-14
+1257	-127	257	77.1	257.3	jx	2197-06-15 01:01:01.111111111	jx   	jx	jx	{"jx":"jx"}	[1257,1258]	{"c1":1257,"c2":"j"}	2197-06-15
+1258	-126	258	77.4	258.3	jy	2198-07-16 01:01:01.111111111	jy   	jy	jy	{"jy":"jy"}	[1258,1259]	{"c1":1258,"c2":"j"}	2198-07-16
+1259	-125	259	77.7	259.3	jz	2199-08-17 01:01:01.111111111	jz   	jz	jz	{"jz":"jz"}	[1259,1260]	{"c1":1259,"c2":"k"}	2199-08-17
+1260	-124	260	78.0	260.3	ka	2200-09-18 01:01:01.111111111	ka   	ka	ka	{"ka":"ka"}	[1260,1261]	{"c1":1260,"c2":"k"}	2200-09-18
+1261	-123	261	78.3	261.3	kb	2201-10-19 01:01:01.111111111	kb   	kb	kb	{"kb":"kb"}	[1261,1262]	{"c1":1261,"c2":"k"}	2201-10-19
+1262	-122	262	78.6	262.3	kc	2202-11-20 01:01:01.111111111	kc   	kc	kc	{"kc":"kc"}	[1262,1263]	{"c1":1262,"c2":"k"}	2202-11-20
+1263	-121	263	78.9	263.3	kd	2203-12-21 01:01:01.111111111	kd   	kd	kd	{"kd":"kd"}	[1263,1264]	{"c1":1263,"c2":"k"}	2203-12-21
+1264	-120	264	79.2	264.3	ke	2204-01-22 01:01:01.111111111	ke   	ke	ke	{"ke":"ke"}	[1264,1265]	{"c1":1264,"c2":"k"}	2204-01-22
+1265	-119	265	79.5	265.3	kf	2205-02-23 01:01:01.111111111	kf   	kf	kf	{"kf":"kf"}	[1265,1266]	{"c1":1265,"c2":"k"}	2205-02-23
+1266	-118	266	79.8	266.3	kg	2206-03-24 01:01:01.111111111	kg   	kg	kg	{"kg":"kg"}	[1266,1267]	{"c1":1266,"c2":"k"}	2206-03-24
+1267	-117	267	80.1	267.3	kh	2207-04-25 01:01:01.111111111	kh   	kh	kh	{"kh":"kh"}	[1267,1268]	{"c1":1267,"c2":"k"}	2207-04-25
+1268	-116	268	80.4	268.3	ki	2208-05-26 01:01:01.111111111	ki   	ki	ki	{"ki":"ki"}	[1268,1269]	{"c1":1268,"c2":"k"}	2208-05-26
+1269	-115	269	80.7	269.3	kj	2209-06-27 01:01:01.111111111	kj   	kj	kj	{"kj":"kj"}	[1269,1270]	{"c1":1269,"c2":"k"}	2209-06-27
+1270	-114	270	81.0	270.3	kk	2210-07-01 01:01:01.111111111	kk   	kk	kk	{"kk":"kk"}	[1270,1271]	{"c1":1270,"c2":"k"}	2210-07-01
+1271	-113	271	81.3	271.3	kl	2211-08-02 01:01:01.111111111	kl   	kl	kl	{"kl":"kl"}	[1271,1272]	{"c1":1271,"c2":"k"}	2211-08-02
+1272	-112	272	81.6	272.3	km	2212-09-03 01:01:01.111111111	km   	km	km	{"km":"km"}	[1272,1273]	{"c1":1272,"c2":"k"}	2212-09-03
+1273	-111	273	81.9	273.3	kn	2213-10-04 01:01:01.111111111	kn   	kn	kn	{"kn":"kn"}	[1273,1274]	{"c1":1273,"c2":"k"}	2213-10-04
+1274	-110	274	82.2	274.3	ko	2214-11-05 01:01:01.111111111	ko   	ko	ko	{"ko":"ko"}	[1274,1275]	{"c1":1274,"c2":"k"}	2214-11-05
+1275	-109	275	82.5	275.3	kp	2215-12-06 01:01:01.111111111	kp   	kp	kp	{"kp":"kp"}	[1275,1276]	{"c1":1275,"c2":"k"}	2215-12-06
+1276	-108	276	82.8	276.3	kq	2216-01-07 01:01:01.111111111	kq   	kq	kq	{"kq":"kq"}	[1276,1277]	{"c1":1276,"c2":"k"}	2216-01-07
+1277	-107	277	83.1	277.3	kr	2217-02-08 01:01:01.111111111	kr   	kr	kr	{"kr":"kr"}	[1277,1278]	{"c1":1277,"c2":"k"}	2217-02-08
+1278	-106	278	83.4	278.3	ks	2218-03-09 01:01:01.111111111	ks   	ks	ks	{"ks":"ks"}	[1278,1279]	{"c1":1278,"c2":"k"}	2218-03-09
+1279	-105	279	83.7	279.3	kt	2219-04-10 01:01:01.111111111	kt   	kt	kt	{"kt":"kt"}	[1279,1280]	{"c1":1279,"c2":"k"}	2219-04-10
+1280	-104	280	84.0	280.3	ku	2220-05-11 01:01:01.111111111	ku   	ku	ku	{"ku":"ku"}	[1280,1281]	{"c1":1280,"c2":"k"}	2220-05-11
+1281	-103	281	84.3	281.3	kv	2221-06-12 01:01:01.111111111	kv   	kv	kv	{"kv":"kv"}	[1281,1282]	{"c1":1281,"c2":"k"}	2221-06-12
+1282	-102	282	84.6	282.3	kw	2222-07-13 01:01:01.111111111	kw   	kw	kw	{"kw":"kw"}	[1282,1283]	{"c1":1282,"c2":"k"}	2222-07-13
+1283	-101	283	84.9	283.3	kx	2223-08-14 01:01:01.111111111	kx   	kx	kx	{"kx":"kx"}	[1283,1284]	{"c1":1283,"c2":"k"}	2223-08-14
+1284	-100	284	85.2	284.3	ky	2224-09-15 01:01:01.111111111	ky   	ky	ky	{"ky":"ky"}	[1284,1285]	{"c1":1284,"c2":"k"}	2224-09-15
+1285	-99	285	85.5	285.3	kz	2225-10-16 01:01:01.111111111	kz   	kz	kz	{"kz":"kz"}	[1285,1286]	{"c1":1285,"c2":"l"}	2225-10-16
+1286	-98	286	85.8	286.3	la	2226-11-17 01:01:01.111111111	la   	la	la	{"la":"la"}	[1286,1287]	{"c1":1286,"c2":"l"}	2226-11-17
+1287	-97	287	86.1	287.3	lb	2227-12-18 01:01:01.111111111	lb   	lb	lb	{"lb":"lb"}	[1287,1288]	{"c1":1287,"c2":"l"}	2227-12-18
+1288	-96	288	86.4	288.3	lc	2228-01-19 01:01:01.111111111	lc   	lc	lc	{"lc":"lc"}	[1288,1289]	{"c1":1288,"c2":"l"}	2228-01-19
+1289	-95	289	86.7	289.3	ld	2229-02-20 01:01:01.111111111	ld   	ld	ld	{"ld":"ld"}	[1289,1290]	{"c1":1289,"c2":"l"}	2229-02-20
+1290	-94	290	87.0	290.3	le	2230-03-21 01:01:01.111111111	le   	le	le	{"le":"le"}	[1290,1291]	{"c1":1290,"c2":"l"}	2230-03-21
+1291	-93	291	87.3	291.3	lf	2231-04-22 01:01:01.111111111	lf   	lf	lf	{"lf":"lf"}	[1291,1292]	{"c1":1291,"c2":"l"}	2231-04-22
+1292	-92	292	87.6	292.3	lg	2232-05-23 01:01:01.111111111	lg   	lg	lg	{"lg":"lg"}	[1292,1293]	{"c1":1292,"c2":"l"}	2232-05-23
+1293	-91	293	87.9	293.3	lh	2233-06-24 01:01:01.111111111	lh   	lh	lh	{"lh":"lh"}	[1293,1294]	{"c1":1293,"c2":"l"}	2233-06-24
+1294	-90	294	88.2	294.3	li	2234-07-25 01:01:01.111111111	li   	li	li	{"li":"li"}	[1294,1295]	{"c1":1294,"c2":"l"}	2234-07-25
+1295	-89	295	88.5	295.3	lj	2235-08-26 01:01:01.111111111	lj   	lj	lj	{"lj":"lj"}	[1295,1296]	{"c1":1295,"c2":"l"}	2235-08-26
+1296	-88	296	88.8	296.3	lk	2236-09-27 01:01:01.111111111	lk   	lk	lk	{"lk":"lk"}	[1296,1297]	{"c1":1296,"c2":"l"}	2236-09-27
+1297	-87	297	89.1	297.3	ll	2237-10-01 01:01:01.111111111	ll   	ll	ll	{"ll":"ll"}	[1297,1298]	{"c1":1297,"c2":"l"}	2237-10-01
+1298	-86	298	89.4	298.3	lm	2238-11-02 01:01:01.111111111	lm   	lm	lm	{"lm":"lm"}	[1298,1299]	{"c1":1298,"c2":"l"}	2238-11-02
+1299	-85	299	89.7	299.3	ln	2239-12-03 01:01:01.111111111	ln   	ln	ln	{"ln":"ln"}	[1299,1300]	{"c1":1299,"c2":"l"}	2239-12-03
+PREHOOK: query: INSERT OVERWRITE TABLE parquet_types
+SELECT cint, ctinyint, csmallint, cfloat, cdouble, cstring1, t, cchar, cvarchar,
+unhex(cbinary), m1, l1, st1, d FROM parquet_types_staging
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types_staging
+PREHOOK: Output: default@parquet_types
+POSTHOOK: query: INSERT OVERWRITE TABLE parquet_types
+SELECT cint, ctinyint, csmallint, cfloat, cdouble, cstring1, t, cchar, cvarchar,
+unhex(cbinary), m1, l1, st1, d FROM parquet_types_staging
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types_staging
+POSTHOOK: Output: default@parquet_types
+POSTHOOK: Lineage: parquet_types.cbinary EXPRESSION [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cbinary, type:string, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cchar SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cchar, type:char(5), comment:null), ]
+POSTHOOK: Lineage: parquet_types.cdouble SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cfloat SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cint SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: parquet_types.csmallint SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:csmallint, type:smallint, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cstring1 SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cstring1, type:string, comment:null), ]
+POSTHOOK: Lineage: parquet_types.ctinyint SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:ctinyint, type:tinyint, comment:null), ]
+POSTHOOK: Lineage: parquet_types.cvarchar SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:cvarchar, type:varchar(10), comment:null), ]
+POSTHOOK: Lineage: parquet_types.d SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:d, type:date, comment:null), ]
+POSTHOOK: Lineage: parquet_types.l1 SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:l1, type:array<int>, comment:null), ]
+POSTHOOK: Lineage: parquet_types.m1 SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:m1, type:map<string,varchar(3)>, comment:null), ]
+POSTHOOK: Lineage: parquet_types.st1 SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:st1, type:struct<c1:int,c2:char(1)>, comment:null), ]
+POSTHOOK: Lineage: parquet_types.t SIMPLE [(parquet_types_staging)parquet_types_staging.FieldSchema(name:t, type:timestamp, comment:null), ]
+PREHOOK: query: -- test types in group by
+
+EXPLAIN SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- test types in group by
+
+EXPLAIN SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: ctinyint (type: tinyint), cint (type: int), csmallint (type: smallint), cstring1 (type: string), cfloat (type: float), cdouble (type: double)
+              outputColumnNames: ctinyint, cint, csmallint, cstring1, cfloat, cdouble
+              Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: max(cint), min(csmallint), count(cstring1), avg(cfloat), stddev_pop(cdouble)
+                keys: ctinyint (type: tinyint)
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: tinyint)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: tinyint)
+                  Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: struct<count:bigint,sum:double,input:float>), _col5 (type: struct<count:bigint,sum:double,variance:double>)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: max(VALUE._col0), min(VALUE._col1), count(VALUE._col2), avg(VALUE._col3), stddev_pop(VALUE._col4)
+          keys: KEY._col0 (type: tinyint)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: tinyint), _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), round(_col4, 5) (type: double), round(_col5, 5) (type: double)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+            Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: tinyint)
+              sort order: +
+              Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: int), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: double), _col5 (type: double)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: tinyint), VALUE._col0 (type: int), VALUE._col1 (type: smallint), VALUE._col2 (type: bigint), VALUE._col3 (type: double), VALUE._col4 (type: double)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  ROUND(AVG(cfloat), 5),
+  ROUND(STDDEV_POP(cdouble),5)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+-128	1256	0	2	38.4	128.0
+-127	1257	1	2	38.7	128.0
+-126	1258	2	2	39.0	128.0
+-125	1259	3	2	39.3	128.0
+-124	1260	4	2	39.6	128.0
+-123	1261	5	2	39.9	128.0
+-122	1262	6	2	40.2	128.0
+-121	1263	7	2	40.5	128.0
+-120	1264	8	2	40.8	128.0
+-119	1265	9	2	41.1	128.0
+-118	1266	10	2	41.4	128.0
+-117	1267	11	2	41.7	128.0
+-116	1268	12	2	42.0	128.0
+-115	1269	13	2	42.3	128.0
+-114	1270	14	2	42.6	128.0
+-113	1271	15	2	42.9	128.0
+-112	1272	16	2	43.2	128.0
+-111	1273	17	2	43.5	128.0
+-110	1274	18	2	43.8	128.0
+-109	1275	19	2	44.1	128.0
+-108	1276	20	2	44.4	128.0
+-107	1277	21	2	44.7	128.0
+-106	1278	22	2	45.0	128.0
+-105	1279	23	2	45.3	128.0
+-104	1280	24	2	45.6	128.0
+-103	1281	25	2	45.9	128.0
+-102	1282	26	2	46.2	128.0
+-101	1283	27	2	46.5	128.0
+-100	1284	28	2	46.8	128.0
+-99	1285	29	2	47.1	128.0
+-98	1286	30	2	47.4	128.0
+-97	1287	31	2	47.7	128.0
+-96	1288	32	2	48.0	128.0
+-95	1289	33	2	48.3	128.0
+-94	1290	34	2	48.6	128.0
+-93	1291	35	2	48.9	128.0
+-92	1292	36	2	49.2	128.0
+-91	1293	37	2	49.5	128.0
+-90	1294	38	2	49.8	128.0
+-89	1295	39	2	50.1	128.0
+-88	1296	40	2	50.4	128.0
+-87	1297	41	2	50.7	128.0
+-86	1298	42	2	51.0	128.0
+-85	1299	43	2	51.3	128.0
+-84	1044	44	1	13.2	0.0
+-83	1045	45	1	13.5	0.0
+-82	1046	46	1	13.8	0.0
+-81	1047	47	1	14.1	0.0
+-80	1048	48	1	14.4	0.0
+-79	1049	49	1	14.7	0.0
+-78	1050	50	1	15.0	0.0
+-77	1051	51	1	15.3	0.0
+-76	1052	52	1	15.6	0.0
+-75	1053	53	1	15.9	0.0
+-74	1054	54	1	16.2	0.0
+-73	1055	55	1	16.5	0.0
+-72	1056	56	1	16.8	0.0
+-71	1057	57	1	17.1	0.0
+-70	1058	58	1	17.4	0.0
+-69	1059	59	1	17.7	0.0
+-68	1060	60	1	18.0	0.0
+-67	1061	61	1	18.3	0.0
+-66	1062	62	1	18.6	0.0
+-65	1063	63	1	18.9	0.0
+-64	1064	64	1	19.2	0.0
+-63	1065	65	1	19.5	0.0
+-62	1066	66	1	19.8	0.0
+-61	1067	67	1	20.1	0.0
+-60	1068	68	1	20.4	0.0
+-59	1069	69	1	20.7	0.0
+-58	1070	70	1	21.0	0.0
+-57	1071	71	1	21.3	0.0
+-56	1072	72	1	21.6	0.0
+-55	1073	73	1	21.9	0.0
+-54	1074	74	1	22.2	0.0
+-53	1075	75	1	22.5	0.0
+-52	1076	76	1	22.8	0.0
+-51	1077	77	1	23.1	0.0
+-50	1078	78	1	23.4	0.0
+-49	1079	79	1	23.7	0.0
+-48	1080	80	1	24.0	0.0
+-47	1081	81	1	24.3	0.0
+-46	1082	82	1	24.6	0.0
+-45	1083	83	1	24.9	0.0
+-44	1084	84	1	25.2	0.0
+-43	1085	85	1	25.5	0.0
+-42	1086	86	1	25.8	0.0
+-41	1087	87	1	26.1	0.0
+-40	1088	88	1	26.4	0.0
+-39	1089	89	1	26.7	0.0
+-38	1090	90	1	27.0	0.0
+-37	1091	91	1	27.3	0.0
+-36	1092	92	1	27.6	0.0
+-35	1093	93	1	27.9	0.0
+-34	1094	94	1	28.2	0.0
+-33	1095	95	1	28.5	0.0
+-32	1096	96	1	28.8	0.0
+-31	1097	97	1	29.1	0.0
+-30	1098	98	1	29.4	0.0
+-29	1099	99	1	29.7	0.0
+-28	1100	100	1	30.0	0.0
+-27	1101	101	1	30.3	0.0
+-26	1102	102	1	30.6	0.0
+-25	1103	103	1	30.9	0.0
+-24	1104	104	1	31.2	0.0
+-23	1105	105	1	31.5	0.0
+-22	1106	106	1	31.8	0.0
+-21	1107	107	1	32.1	0.0
+-20	1108	108	1	32.4	0.0
+-19	1109	109	1	32.7	0.0
+-18	1110	110	1	33.0	0.0
+-17	1111	111	1	33.3	0.0
+-16	1112	112	1	33.6	0.0
+-15	1113	113	1	33.9	0.0
+-14	1114	114	1	34.2	0.0
+-13	1115	115	1	34.5	0.0
+-12	1116	116	1	34.8	0.0
+-11	1117	117	1	35.1	0.0
+-10	1118	118	1	35.4	0.0
+-9	1119	119	1	35.7	0.0
+-8	1120	120	1	36.0	0.0
+-7	1121	121	1	36.3	0.0
+-6	1122	122	1	36.6	0.0
+-5	1123	123	1	36.9	0.0
+-4	1124	124	1	37.2	0.0
+-3	1125	125	1	37.5	0.0
+-2	1126	126	1	37.8	0.0
+-1	1127	127	1	38.1	0.0
+0	1128	128	1	38.4	0.0
+1	1129	129	1	38.7	0.0
+2	1130	130	1	39.0	0.0
+3	1131	131	1	39.3	0.0
+4	1132	132	1	39.6	0.0
+5	1133	133	1	39.9	0.0
+6	1134	134	1	40.2	0.0
+7	1135	135	1	40.5	0.0
+8	1136	136	1	40.8	0.0
+9	1137	137	1	41.1	0.0
+10	1138	138	1	41.4	0.0
+11	1139	139	1	41.7	0.0
+12	1140	140	1	42.0	0.0
+13	1141	141	1	42.3	0.0
+14	1142	142	1	42.6	0.0
+15	1143	143	1	42.9	0.0
+16	1144	144	1	43.2	0.0
+17	1145	145	1	43.5	0.0
+18	1146	146	1	43.8	0.0
+19	1147	147	1	44.1	0.0
+20	1148	148	1	44.4	0.0
+21	1149	149	1	44.7	0.0
+22	1150	150	1	45.0	0.0
+23	1151	151	1	45.3	0.0
+24	1152	152	1	45.6	0.0
+25	1153	153	1	45.9	0.0
+26	1154	154	1	46.2	0.0
+27	1155	155	1	46.5	0.0
+28	1156	156	1	46.8	0.0
+29	1157	157	1	47.1	0.0
+30	1158	158	1	47.4	0.0
+31	1159	159	1	47.7	0.0
+32	1160	160	1	48.0	0.0
+33	1161	161	1	48.3	0.0
+34	1162	162	1	48.6	0.0
+35	1163	163	1	48.9	0.0
+36	1164	164	1	49.2	0.0
+37	1165	165	1	49.5	0.0
+38	1166	166	1	49.8	0.0
+39	1167	167	1	50.1	0.0
+40	1168	168	1	50.4	0.0
+41	1169	169	1	50.7	0.0
+42	1170	170	1	51.0	0.0
+43	1171	171	1	51.3	0.0
+44	1172	172	1	51.6	0.0
+45	1173	173	1	51.9	0.0
+46	1174	174	1	52.2	0.0
+47	1175	175	1	52.5	0.0
+48	1176	176	1	52.8	0.0
+49	1177	177	1	53.1	0.0
+50	1178	178	1	53.4	0.0
+51	1179	179	1	53.7	0.0
+52	1180	180	1	54.0	0.0
+53	1181	181	1	54.3	0.0
+54	1182	182	1	54.6	0.0
+55	1183	183	1	54.9	0.0
+56	1184	184	1	55.2	0.0
+57	1185	185	1	55.5	0.0
+58	1186	186	1	55.8	0.0
+59	1187	187	1	56.1	0.0
+60	1188	188	1	56.4	0.0
+61	1189	189	1	56.7	0.0
+62	1190	190	1	57.0	0.0
+63	1191	191	1	57.3	0.0
+64	1192	192	1	57.6	0.0
+65	1193	193	1	57.9	0.0
+66	1194	194	1	58.2	0.0
+67	1195	195	1	58.5	0.0
+68	1196	196	1	58.8	0.0
+69	1197	197	1	59.1	0.0
+70	1198	198	1	59.4	0.0
+71	1199	199	1	59.7	0.0
+72	1200	200	1	60.0	0.0
+73	1201	201	1	60.3	0.0
+74	1202	202	1	60.6	0.0
+75	1203	203	1	60.9	0.0
+76	1204	204	1	61.2	0.0
+77	1205	205	1	61.5	0.0
+78	1206	206	1	61.8	0.0
+79	1207	207	1	62.1	0.0
+80	1208	208	1	62.4	0.0
+81	1209	209	1	62.7	0.0
+82	1210	210	1	63.0	0.0
+83	1211	211	1	63.3	0.0
+84	1212	212	1	63.6	0.0
+85	1213	213	1	63.9	0.0
+86	1214	214	1	64.2	0.0
+87	1215	215	1	64.5	0.0
+88	1216	216	1	64.8	0.0
+89	1217	217	1	65.1	0.0
+90	1218	218	1	65.4	0.0
+91	1219	219	1	65.7	0.0
+92	1220	220	1	66.0	0.0
+93	1221	221	1	66.3	0.0
+94	1222	222	1	66.6	0.0
+95	1223	223	1	66.9	0.0
+96	1224	224	1	67.2	0.0
+97	1225	225	1	67.5	0.0
+98	1226	226	1	67.8	0.0
+99	1227	227	1	68.1	0.0
+100	1228	228	1	68.4	0.0
+101	1229	229	1	68.7	0.0
+102	1230	230	1	69.0	0.0
+103	1231	231	1	69.3	0.0
+104	1232	232	1	69.6	0.0
+105	1233	233	1	69.9	0.0
+106	1234	234	1	70.2	0.0
+107	1235	235	1	70.5	0.0
+108	1236	236	1	70.8	0.0
+109	1237	237	1	71.1	0.0
+110	1238	238	1	71.4	0.0
+111	1239	239	1	71.7	0.0
+112	1240	240	1	72.0	0.0
+113	1241	241	1	72.3	0.0
+114	1242	242	1	72.6	0.0
+115	1243	243	1	72.9	0.0
+116	1244	244	1	73.2	0.0
+117	1245	245	1	73.5	0.0
+118	1246	246	1	73.8	0.0
+119	1247	247	1	74.1	0.0
+120	1248	248	1	74.4	0.0
+121	1249	249	1	74.7	0.0
+122	1250	250	1	75.0	0.0
+123	1251	251	1	75.3	0.0
+124	1252	252	1	75.6	0.0
+125	1253	253	1	75.9	0.0
+126	1254	254	1	76.2	0.0
+127	1255	255	1	76.5	0.0
+PREHOOK: query: EXPLAIN SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cfloat (type: float)
+              outputColumnNames: cfloat
+              Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cfloat (type: float)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: float)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: float)
+                  Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: float)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: float)
+              sort order: +
+              Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: float), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cfloat, count(*) FROM parquet_types GROUP BY cfloat ORDER BY cfloat
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+0.0	1
+0.3	1
+0.6	1
+0.9	1
+1.2	1
+1.5	1
+1.8	1
+2.1	1
+2.4	1
+2.7	1
+3.0	1
+3.3	1
+3.6	1
+3.9	1
+4.2	1
+4.5	1
+4.8	1
+5.1	1
+5.4	1
+5.7	1
+6.0	1
+6.3	1
+6.6	1
+6.9	1
+7.2	1
+7.5	1
+7.8	1
+8.1	1
+8.4	1
+8.7	1
+9.0	1
+9.3	1
+9.6	1
+9.9	1
+10.2	1
+10.5	1
+10.8	1
+11.1	1
+11.4	1
+11.7	1
+12.0	1
+12.3	1
+12.6	1
+12.9	1
+13.2	1
+13.5	1
+13.8	1
+14.1	1
+14.4	1
+14.7	1
+15.0	1
+15.3	1
+15.6	1
+15.9	1
+16.2	1
+16.5	1
+16.8	1
+17.1	1
+17.4	1
+17.7	1
+18.0	1
+18.3	1
+18.6	1
+18.9	1
+19.2	1
+19.5	1
+19.8	1
+20.1	1
+20.4	1
+20.7	1
+21.0	1
+21.3	1
+21.6	1
+21.9	1
+22.2	1
+22.5	1
+22.8	1
+23.1	1
+23.4	1
+23.7	1
+24.0	1
+24.3	1
+24.6	1
+24.9	1
+25.2	1
+25.5	1
+25.8	1
+26.1	1
+26.4	1
+26.7	1
+27.0	1
+27.3	1
+27.6	1
+27.9	1
+28.2	1
+28.5	1
+28.8	1
+29.1	1
+29.4	1
+29.7	1
+30.0	1
+30.3	1
+30.6	1
+30.9	1
+31.2	1
+31.5	1
+31.8	1
+32.1	1
+32.4	1
+32.7	1
+33.0	1
+33.3	1
+33.6	1
+33.9	1
+34.2	1
+34.5	1
+34.8	1
+35.1	1
+35.4	1
+35.7	1
+36.0	1
+36.3	1
+36.6	1
+36.9	1
+37.2	1
+37.5	1
+37.8	1
+38.1	1
+38.4	1
+38.7	1
+39.0	1
+39.3	1
+39.6	1
+39.9	1
+40.2	1
+40.5	1
+40.8	1
+41.1	1
+41.4	1
+41.7	1
+42.0	1
+42.3	1
+42.6	1
+42.9	1
+43.2	1
+43.5	1
+43.8	1
+44.1	1
+44.4	1
+44.7	1
+45.0	1
+45.3	1
+45.6	1
+45.9	1
+46.2	1
+46.5	1
+46.8	1
+47.1	1
+47.4	1
+47.7	1
+48.0	1
+48.3	1
+48.6	1
+48.9	1
+49.2	1
+49.5	1
+49.8	1
+50.1	1
+50.4	1
+50.7	1
+51.0	1
+51.3	1
+51.6	1
+51.9	1
+52.2	1
+52.5	1
+52.8	1
+53.1	1
+53.4	1
+53.7	1
+54.0	1
+54.3	1
+54.6	1
+54.9	1
+55.2	1
+55.5	1
+55.8	1
+56.1	1
+56.4	1
+56.7	1
+57.0	1
+57.3	1
+57.6	1
+57.9	1
+58.2	1
+58.5	1
+58.8	1
+59.1	1
+59.4	1
+59.7	1
+60.0	1
+60.3	1
+60.6	1
+60.9	1
+61.2	1
+61.5	1
+61.8	1
+62.1	1
+62.4	1
+62.7	1
+63.0	1
+63.3	1
+63.6	1
+63.9	1
+64.2	1
+64.5	1
+64.8	1
+65.1	1
+65.4	1
+65.7	1
+66.0	1
+66.3	1
+66.6	1
+66.9	1
+67.2	1
+67.5	1
+67.8	1
+68.1	1
+68.4	1
+68.7	1
+69.0	1
+69.3	1
+69.6	1
+69.9	1
+70.2	1
+70.5	1
+70.8	1
+71.1	1
+71.4	1
+71.7	1
+72.0	1
+72.3	1
+72.6	1
+72.9	1
+73.2	1
+73.5	1
+73.8	1
+74.1	1
+74.4	1
+74.7	1
+75.0	1
+75.3	1
+75.6	1
+75.9	1
+76.2	1
+76.5	1
+76.8	1
+77.1	1
+77.4	1
+77.7	1
+78.0	1
+78.3	1
+78.6	1
+78.9	1
+79.2	1
+79.5	1
+79.8	1
+80.1	1
+80.4	1
+80.7	1
+81.0	1
+81.3	1
+81.6	1
+81.9	1
+82.2	1
+82.5	1
+82.8	1
+83.1	1
+83.4	1
+83.7	1
+84.0	1
+84.3	1
+84.6	1
+84.9	1
+85.2	1
+85.5	1
+85.8	1
+86.1	1
+86.4	1
+86.7	1
+87.0	1
+87.3	1
+87.6	1
+87.9	1
+88.2	1
+88.5	1
+88.8	1
+89.1	1
+89.4	1
+89.7	1
+PREHOOK: query: EXPLAIN SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cchar (type: char(5))
+              outputColumnNames: cchar
+              Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cchar (type: char(5))
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: char(5))
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: char(5))
+                  Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: char(5))
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: char(5))
+              sort order: +
+              Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: char(5)), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cchar, count(*) FROM parquet_types GROUP BY cchar ORDER BY cchar
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+     	1
+b    	1
+ba   	1
+bb   	1
+bc   	1
+bd   	1
+be   	1
+bf   	1
+bg   	1
+bh   	1
+bi   	1
+bj   	1
+bk   	1
+bl   	1
+bm   	1
+bn   	1
+bo   	1
+bp   	1
+bq   	1
+br   	1
+bs   	1
+bt   	1
+bu   	1
+bv   	1
+bw   	1
+bx   	1
+by   	1
+bz   	1
+c    	1
+ca   	1
+cb   	1
+cc   	1
+cd   	1
+ce   	1
+cf   	1
+cg   	1
+ch   	1
+ci   	1
+cj   	1
+ck   	1
+cl   	1
+cm   	1
+cn   	1
+co   	1
+cp   	1
+cq   	1
+cr   	1
+cs   	1
+ct   	1
+cu   	1
+cv   	1
+cw   	1
+cx   	1
+cy   	1
+cz   	1
+d    	1
+da   	1
+db   	1
+dc   	1
+dd   	1
+de   	1
+df   	1
+dg   	1
+dh   	1
+di   	1
+dj   	1
+dk   	1
+dl   	1
+dm   	1
+dn   	1
+do   	1
+dp   	1
+dq   	1
+dr   	1
+ds   	1
+dt   	1
+du   	1
+dv   	1
+dw   	1
+dx   	1
+dy   	1
+dz   	1
+e    	1
+ea   	1
+eb   	1
+ec   	1
+ed   	1
+ee   	1
+ef   	1
+eg   	1
+eh   	1
+ei   	1
+ej   	1
+ek   	1
+el   	1
+em   	1
+en   	1
+eo   	1
+ep   	1
+eq   	1
+er   	1
+es   	1
+et   	1
+eu   	1
+ev   	1
+ew   	1
+ex   	1
+ey   	1
+ez   	1
+f    	1
+fa   	1
+fb   	1
+fc   	1
+fd   	1
+fe   	1
+ff   	1
+fg   	1
+fh   	1
+fi   	1
+fj   	1
+fk   	1
+fl   	1
+fm   	1
+fn   	1
+fo   	1
+fp   	1
+fq   	1
+fr   	1
+fs   	1
+ft   	1
+fu   	1
+fv   	1
+fw   	1
+fx   	1
+fy   	1
+fz   	1
+g    	1
+ga   	1
+gb   	1
+gc   	1
+gd   	1
+ge   	1
+gf   	1
+gg   	1
+gh   	1
+gi   	1
+gj   	1
+gk   	1
+gl   	1
+gm   	1
+gn   	1
+go   	1
+gp   	1
+gq   	1
+gr   	1
+gs   	1
+gt   	1
+gu   	1
+gv   	1
+gw   	1
+gx   	1
+gy   	1
+gz   	1
+h    	1
+ha   	1
+hb   	1
+hc   	1
+hd   	1
+he   	1
+hf   	1
+hg   	1
+hh   	1
+hi   	1
+hj   	1
+hk   	1
+hl   	1
+hm   	1
+hn   	1
+ho   	1
+hp   	1
+hq   	1
+hr   	1
+hs   	1
+ht   	1
+hu   	1
+hv   	1
+hw   	1
+hx   	1
+hy   	1
+hz   	1
+i    	1
+ia   	1
+ib   	1
+ic   	1
+id   	1
+ie   	1
+if   	1
+ig   	1
+ih   	1
+ii   	1
+ij   	1
+ik   	1
+il   	1
+im   	1
+in   	1
+io   	1
+ip   	1
+iq   	1
+ir   	1
+is   	1
+it   	1
+iu   	1
+iv   	1
+iw   	1
+ix   	1
+iy   	1
+iz   	1
+j    	1
+ja   	1
+jb   	1
+jc   	1
+jd   	1
+je   	1
+jf   	1
+jg   	1
+jh   	1
+ji   	1
+jj   	1
+jk   	1
+jl   	1
+jm   	1
+jn   	1
+jo   	1
+jp   	1
+jq   	1
+jr   	1
+js   	1
+jt   	1
+ju   	1
+jv   	1
+jw   	1
+jx   	1
+jy   	1
+jz   	1
+k    	1
+ka   	1
+kb   	1
+kc   	1
+kd   	1
+ke   	1
+kf   	1
+kg   	1
+kh   	1
+ki   	1
+kj   	1
+kk   	1
+kl   	1
+km   	1
+kn   	1
+ko   	1
+kp   	1
+kq   	1
+kr   	1
+ks   	1
+kt   	1
+ku   	1
+kv   	1
+kw   	1
+kx   	1
+ky   	1
+kz   	1
+l    	1
+la   	1
+lb   	1
+lc   	1
+ld   	1
+le   	1
+lf   	1
+lg   	1
+lh   	1
+li   	1
+lj   	1
+lk   	1
+ll   	1
+lm   	1
+ln   	1
+m    	1
+n    	1
+o    	1
+p    	1
+q    	1
+r    	1
+s    	1
+t    	1
+u    	1
+v    	1
+w    	1
+x    	1
+y    	1
+z    	1
+PREHOOK: query: EXPLAIN SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cvarchar (type: varchar(10))
+              outputColumnNames: cvarchar
+              Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cvarchar (type: varchar(10))
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: varchar(10))
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: varchar(10))
+                  Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: varchar(10))
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: varchar(10))
+              sort order: +
+              Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: varchar(10)), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cvarchar, count(*) FROM parquet_types GROUP BY cvarchar ORDER BY cvarchar
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+	1
+b	1
+ba	1
+bb	1
+bc	1
+bd	1
+be	1
+bf	1
+bg	1
+bh	1
+bi	1
+bj	1
+bk	1
+bl	1
+bm	1
+bn	1
+bo	1
+bp	1
+bq	1
+br	1
+bs	1
+bt	1
+bu	1
+bv	1
+bw	1
+bx	1
+by	1
+bz	1
+c	1
+ca	1
+cb	1
+cc	1
+cd	1
+ce	1
+cf	1
+cg	1
+ch	1
+ci	1
+cj	1
+ck	1
+cl	1
+cm	1
+cn	1
+co	1
+cp	1
+cq	1
+cr	1
+cs	1
+ct	1
+cu	1
+cv	1
+cw	1
+cx	1
+cy	1
+cz	1
+d	1
+da	1
+db	1
+dc	1
+dd	1
+de	1
+df	1
+dg	1
+dh	1
+di	1
+dj	1
+dk	1
+dl	1
+dm	1
+dn	1
+do	1
+dp	1
+dq	1
+dr	1
+ds	1
+dt	1
+du	1
+dv	1
+dw	1
+dx	1
+dy	1
+dz	1
+e	1
+ea	1
+eb	1
+ec	1
+ed	1
+ee	1
+ef	1
+eg	1
+eh	1
+ei	1
+ej	1
+ek	1
+el	1
+em	1
+en	1
+eo	1
+ep	1
+eq	1
+er	1
+es	1
+et	1
+eu	1
+ev	1
+ew	1
+ex	1
+ey	1
+ez	1
+f	1
+fa	1
+fb	1
+fc	1
+fd	1
+fe	1
+ff	1
+fg	1
+fh	1
+fi	1
+fj	1
+fk	1
+fl	1
+fm	1
+fn	1
+fo	1
+fp	1
+fq	1
+fr	1
+fs	1
+ft	1
+fu	1
+fv	1
+fw	1
+fx	1
+fy	1
+fz	1
+g	1
+ga	1
+gb	1
+gc	1
+gd	1
+ge	1
+gf	1
+gg	1
+gh	1
+gi	1
+gj	1
+gk	1
+gl	1
+gm	1
+gn	1
+go	1
+gp	1
+gq	1
+gr	1
+gs	1
+gt	1
+gu	1
+gv	1
+gw	1
+gx	1
+gy	1
+gz	1
+h	1
+ha	1
+hb	1
+hc	1
+hd	1
+he	1
+hf	1
+hg	1
+hh	1
+hi	1
+hj	1
+hk	1
+hl	1
+hm	1
+hn	1
+ho	1
+hp	1
+hq	1
+hr	1
+hs	1
+ht	1
+hu	1
+hv	1
+hw	1
+hx	1
+hy	1
+hz	1
+i	1
+ia	1
+ib	1
+ic	1
+id	1
+ie	1
+if	1
+ig	1
+ih	1
+ii	1
+ij	1
+ik	1
+il	1
+im	1
+in	1
+io	1
+ip	1
+iq	1
+ir	1
+is	1
+it	1
+iu	1
+iv	1
+iw	1
+ix	1
+iy	1
+iz	1
+j	1
+ja	1
+jb	1
+jc	1
+jd	1
+je	1
+jf	1
+jg	1
+jh	1
+ji	1
+jj	1
+jk	1
+jl	1
+jm	1
+jn	1
+jo	1
+jp	1
+jq	1
+jr	1
+js	1
+jt	1
+ju	1
+jv	1
+jw	1
+jx	1
+jy	1
+jz	1
+k	1
+ka	1
+kb	1
+kc	1
+kd	1
+ke	1
+kf	1
+kg	1
+kh	1
+ki	1
+kj	1
+kk	1
+kl	1
+km	1
+kn	1
+ko	1
+kp	1
+kq	1
+kr	1
+ks	1
+kt	1
+ku	1
+kv	1
+kw	1
+kx	1
+ky	1
+kz	1
+l	1
+la	1
+lb	1
+lc	1
+ld	1
+le	1
+lf	1
+lg	1
+lh	1
+li	1
+lj	1
+lk	1
+ll	1
+lm	1
+ln	1
+m	1
+n	1
+o	1
+p	1
+q	1
+r	1
+s	1
+t	1
+u	1
+v	1
+w	1
+x	1
+y	1
+z	1
+PREHOOK: query: EXPLAIN SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: parquet_types
+            Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cstring1 (type: string)
+              outputColumnNames: cstring1
+              Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cstring1 (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 300 Data size: 4200 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: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: string)
+              sort order: +
+              Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cstring1, count(*) FROM parquet_types GROUP BY cstring1 ORDER BY cstring1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+	1
+b	1
+ba	1
+bb	1
+bc	1
+bd	1
+be	1
+bf	1
+bg	1
+bh	1
+bi	1
+bj	1
+bk	1
+bl	1
+bm	1
+bn	1
+bo	1
+bp	1
+bq	1
+br	1
+bs	1
+bt	1
+bu	1
+bv	1
+bw	1
+bx	1
+by	1
+bz	1
+c	1
+ca	1
+cb	1
+cc	1
+cd	1
+ce	1
+cf	1
+cg	1
+ch	1
+ci	1
+cj	1
+ck	1
+cl	1
+cm	1
+cn	1
+co	1
+cp	1
+cq	1
+cr	1
+cs	1
+ct	1
+cu	1
+cv	1
+cw	1
+cx	1
+cy	1
+cz	1
+d	1
+da	1
+db	1
+dc	1
+dd	1
+de	1
+df	1
+dg	1
+dh	1
+di	1
+dj	1
+dk	1
+dl	1
+dm	1
+dn	1
+do	1
+dp	1
+dq	1
+dr	1
+ds	1
+dt	1
+du	1
+dv	1
+dw	1
+dx	1
+dy	1
+dz	1
+e	1
+ea	1
+eb	1
+ec	1
+ed	1
+ee	1
+ef	1
+eg	1
+eh	1
+ei	1
+ej	1
+ek	1
+el	1
+em	1
+en	1
+eo	1
+ep	1
+eq	1
+er	1
+es	1
+et	1
+eu	1
+ev	1
+ew	1
+ex	1
+ey	1
+ez	1
+f	1
+fa	1
+fb	1
+fc	1
+fd	1
+fe	1
+ff	1
+fg	1
+fh	1
+fi	1
+fj	1
+fk	1
+fl	1
+fm	1
+fn	1
+fo	1
+fp	1
+fq	1
+fr	1
+fs	1
+ft	1
+fu	1
+fv	1
+fw	1
+fx	1
+fy	1
+fz	1
+g	1
+ga	1
+gb	1
+gc	1
+gd	1
+ge	1
+gf	1
+gg	1
+gh	1
+gi	1
+gj	1
+gk	1
+gl	1
+gm	1
+gn	1
+go	1
+gp	1
+gq	1
+gr	1
+gs	1
+gt	1
+gu	1
+gv	1
+gw	1
+gx	1
+gy	1
+gz	1
+h	1
+ha	1
+hb	1
+hc	1
+hd	1
+he	1
+hf	1
+hg	1
+hh	1
+hi	1
+hj	1
+hk	1
+hl	1
+hm	1
+hn	1
+ho	1
+hp	1
+hq	1
+hr	1
+hs	1
+ht	1
+hu	1
+hv	1
+hw	1
+hx	1
+hy	1
+hz	1
+i	1
+ia	1
+ib	1
+ic	1
+id	1
+ie	1
+if	1
+ig	1
+ih	1
+ii	1
+ij	1
+ik	1
+il	1
+im	1
+in	1
+io	1
+ip	1
+iq	1
+ir	1
+is	1
+it	1
+iu	1
+iv	1
+iw	1
+ix	1
+iy	1
+iz	1
+j	1
+ja	1
+jb	1
+jc	1
+jd	1
+je	1
+jf	1
+jg	1
+jh	1
+ji	1
+jj	1
+jk	1
+jl	1
+jm	1
+jn	1
+jo	1
+jp	1
+jq	1
+jr	1
+js	1
+jt	1
+ju	1
+jv	1
+jw	1
+jx	1
+jy	1
+jz	1
+k	1
+ka	1
+kb	1
+kc	1
+kd	1
+ke	1
+kf	1
+kg	1
+kh	1
+ki	1
+kj	1
+kk	1
+kl	1
+km	1
+kn	1
+ko	1
+kp	1
+kq	1
+kr	1
+ks	1
+kt	1
+ku	1
+kv	1
+kw	1
+kx	1
+ky	1
+kz	1
+l	1
+la	1
+lb	1
+lc	1
+ld	1
+le	1
+lf	1
+lg	1
+lh	1
+li	1
+lj	1
+lk	1
+ll	1
+lm	1
+ln	1
+m	1
+n	1
+o	1
+p	1
+q	1
+r	1
+s	1
+t	1
+u	1
+v	1
+w	1
+x	1
+y	1
+z	1
+PREHOOK: query: EXPLAIN SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary
+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: parquet_types
+            Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: cbinary (type: binary)
+              outputColumnNames: cbinary
+              Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                aggregations: count()
+                keys: cbinary (type: binary)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: binary)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: binary)
+                  Statistics: Num rows: 300 Data size: 4200 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          keys: KEY._col0 (type: binary)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: hex(_col0) (type: string), _col1 (type: bigint)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 150 Data size: 2100 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary
+PREHOOK: type: QUERY
+PREHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT hex(cbinary), count(*) FROM parquet_types GROUP BY cbinary
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@parquet_types
+#### A masked pattern was here ####
+NULL	264
+	1
+0B	1
+0C	1
+0D	1
+0E	1
+0F	1
+BA	1
+BB	1
+BC	1
+BD	1
+BE	1
+BF	1
+CA	1
+CB	1
+CC	1
+CD	1
+CE	1
+CF	1
+DA	1
+DB	1
+DC	1
+DD	1
+DE	1
+DF	1
+EA	1
+EB	1
+EC	1
+ED	1
+EE	1
+EF	1
+FA	1
+FB	1
+FC	1
+FD	1
+FE	1
+FF	1


[22/35] hive git commit: HIVE-15148 : disallow loading data into bucketed tables (by default) (Sergey Shelukhin, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-15148 : disallow loading data into bucketed tables (by default) (Sergey Shelukhin, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: 0a6fc63858601d30ba7b6da61663e82e300e454d
Parents: 8a1dcd7
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri Nov 18 11:14:20 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri Nov 18 11:14:20 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  19 +-
 data/scripts/q_test_init.sql                    |  20 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |  10 +
 .../alter_concatenate_indexed_table.q           |   2 +
 .../clientnegative/alter_view_failure6.q        |   2 +
 .../clientnegative/bucket_mapjoin_mismatch1.q   |   2 +
 .../bucket_mapjoin_wrong_table_metadata_1.q     |   2 +
 .../bucket_mapjoin_wrong_table_metadata_2.q     |   2 +
 .../clientnegative/compare_double_bigint.q      |   2 +
 .../clientnegative/compare_string_bigint.q      |   2 +
 .../dynamic_partitions_with_whitelist.q         |   2 +
 .../exim_11_nonpart_noncompat_sorting.q         |   2 +
 ql/src/test/queries/clientnegative/input4.q     |   2 +
 .../queries/clientnegative/input_part0_neg.q    |   2 +
 .../insertover_dynapart_ifnotexists.q           |   2 +
 .../test/queries/clientnegative/strict_join.q   |   2 +
 .../queries/clientnegative/strict_orderby.q     |   2 +
 .../queries/clientnegative/strict_pruning.q     |   2 +
 .../alter_concatenate_indexed_table.q           |   2 +
 .../test/queries/clientpositive/alter_merge.q   |   2 +
 .../test/queries/clientpositive/alter_merge_2.q |   2 +
 .../queries/clientpositive/alter_merge_stats.q  |   2 +
 .../queries/clientpositive/autoColumnStats_1.q  |   2 +
 .../queries/clientpositive/autoColumnStats_2.q  |   2 +
 .../clientpositive/auto_sortmerge_join_1.q      |   2 +
 .../clientpositive/auto_sortmerge_join_11.q     |   2 +
 .../clientpositive/auto_sortmerge_join_12.q     |   2 +
 .../clientpositive/auto_sortmerge_join_16.q     |   2 +
 .../clientpositive/auto_sortmerge_join_2.q      |   2 +
 .../clientpositive/auto_sortmerge_join_3.q      |   2 +
 .../clientpositive/auto_sortmerge_join_4.q      |   2 +
 .../clientpositive/auto_sortmerge_join_5.q      |   2 +
 .../clientpositive/auto_sortmerge_join_7.q      |   2 +
 .../clientpositive/auto_sortmerge_join_8.q      |   2 +
 .../queries/clientpositive/bucket_map_join_1.q  |   2 +
 .../queries/clientpositive/bucket_map_join_2.q  |   2 +
 .../clientpositive/bucket_map_join_spark1.q     |   2 +
 .../clientpositive/bucket_map_join_spark2.q     |   2 +
 .../clientpositive/bucket_map_join_spark3.q     |   2 +
 .../clientpositive/bucket_map_join_tez1.q       |   2 +
 .../clientpositive/bucket_map_join_tez2.q       |   2 +
 .../queries/clientpositive/bucketcontext_1.q    |   2 +
 .../queries/clientpositive/bucketcontext_2.q    |   2 +
 .../queries/clientpositive/bucketcontext_3.q    |   2 +
 .../queries/clientpositive/bucketcontext_4.q    |   2 +
 .../queries/clientpositive/bucketcontext_5.q    |   2 +
 .../queries/clientpositive/bucketcontext_6.q    |   2 +
 .../queries/clientpositive/bucketcontext_7.q    |   2 +
 .../queries/clientpositive/bucketcontext_8.q    |   2 +
 .../bucketizedhiveinputformat_auto.q            |   2 +
 .../queries/clientpositive/bucketmapjoin1.q     |   2 +
 .../queries/clientpositive/bucketmapjoin10.q    |   2 +
 .../queries/clientpositive/bucketmapjoin11.q    |   2 +
 .../queries/clientpositive/bucketmapjoin12.q    |   2 +
 .../queries/clientpositive/bucketmapjoin2.q     |   2 +
 .../queries/clientpositive/bucketmapjoin3.q     |   2 +
 .../queries/clientpositive/bucketmapjoin4.q     |   2 +
 .../queries/clientpositive/bucketmapjoin5.q     |   2 +
 .../queries/clientpositive/bucketmapjoin7.q     |   2 +
 .../queries/clientpositive/bucketmapjoin8.q     |   2 +
 .../queries/clientpositive/bucketmapjoin9.q     |   2 +
 .../clientpositive/bucketmapjoin_negative.q     |   2 +
 .../clientpositive/bucketmapjoin_negative2.q    |   2 +
 .../clientpositive/bucketmapjoin_negative3.q    |   2 +
 .../cbo_rp_udaf_percentile_approx_23.q          |   2 +
 ql/src/test/queries/clientpositive/char_1.q     |   2 +
 ql/src/test/queries/clientpositive/char_serde.q |   2 +
 .../clientpositive/create_merge_compressed.q    |   2 +
 .../queries/clientpositive/explainanalyze_2.q   |   2 +
 .../queries/clientpositive/explainanalyze_3.q   |   2 +
 .../test/queries/clientpositive/explainuser_1.q |   2 +
 .../test/queries/clientpositive/explainuser_2.q |   2 +
 .../test/queries/clientpositive/explainuser_3.q |   2 +
 .../test/queries/clientpositive/global_limit.q  |   2 +
 .../clientpositive/infer_bucket_sort_dyn_part.q |   2 +
 .../queries/clientpositive/insert_orig_table.q  |   2 +
 .../clientpositive/insert_values_orig_table.q   |   2 +
 .../insert_values_orig_table_use_metadata.q     |   2 +
 .../clientpositive/merge_dynamic_partition.q    |   2 +
 .../clientpositive/merge_dynamic_partition2.q   |   2 +
 .../clientpositive/merge_dynamic_partition3.q   |   2 +
 .../clientpositive/merge_dynamic_partition4.q   |   2 +
 .../clientpositive/merge_dynamic_partition5.q   |   2 +
 ql/src/test/queries/clientpositive/mergejoin.q  |   2 +
 .../test/queries/clientpositive/smb_mapjoin_1.q |   2 +
 .../queries/clientpositive/smb_mapjoin_10.q     |   2 +
 .../test/queries/clientpositive/smb_mapjoin_2.q |   2 +
 .../queries/clientpositive/smb_mapjoin_25.q     |   2 +
 .../test/queries/clientpositive/smb_mapjoin_3.q |   2 +
 .../test/queries/clientpositive/smb_mapjoin_4.q |   2 +
 .../test/queries/clientpositive/smb_mapjoin_5.q |   2 +
 .../test/queries/clientpositive/smb_mapjoin_7.q |   2 +
 .../test/queries/clientpositive/smb_mapjoin_8.q |   2 +
 ql/src/test/queries/clientpositive/stats1.q     |   2 +
 ql/src/test/queries/clientpositive/stats11.q    |   2 +
 ql/src/test/queries/clientpositive/stats18.q    |   2 +
 ql/src/test/queries/clientpositive/tez_fsstat.q |   2 +
 ql/src/test/queries/clientpositive/tez_smb_1.q  |   2 +
 .../test/queries/clientpositive/tez_smb_empty.q |   2 +
 .../test/queries/clientpositive/tez_smb_main.q  |   2 +
 .../clientpositive/udaf_percentile_approx_20.q  |   2 +
 .../clientpositive/udaf_percentile_approx_23.q  |   2 +
 ql/src/test/queries/clientpositive/varchar_1.q  |   2 +
 .../test/queries/clientpositive/varchar_serde.q |   2 +
 .../clientnegative/alter_view_failure6.q.out    |   2 +-
 .../clientnegative/compare_double_bigint.q.out  |   2 +-
 .../clientnegative/compare_string_bigint.q.out  |   2 +-
 ql/src/test/results/clientnegative/input4.q.out |   2 +-
 .../clientnegative/input_part0_neg.q.out        |   2 +-
 .../results/clientnegative/strict_join.q.out    |   2 +-
 .../results/clientnegative/strict_orderby.q.out |   2 +-
 .../results/clientnegative/strict_pruning.q.out |   2 +-
 .../test/results/clientpositive/sample2.q.out   | 934 +++++++++----------
 .../test/results/clientpositive/sample4.q.out   | 934 +++++++++----------
 .../test/results/clientpositive/sample6.q.out   | 701 +++++++++-----
 .../test/results/clientpositive/sample7.q.out   | 362 +++----
 .../test/results/clientpositive/sample9.q.out   | 934 +++++++++----------
 .../results/clientpositive/spark/sample2.q.out  |   4 +-
 .../results/clientpositive/spark/sample4.q.out  |   4 +-
 .../results/clientpositive/spark/sample6.q.out  | 249 ++++-
 .../results/clientpositive/spark/sample7.q.out  |   4 +-
 121 files changed, 2537 insertions(+), 1856 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/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 200a1ad..a9474c4 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1024,8 +1024,12 @@ public class HiveConf extends Configuration {
         "  Comparing bigints and strings.\n" +
         "  Comparing bigints and doubles."),
     HIVE_STRICT_CHECKS_CARTESIAN("hive.strict.checks.cartesian.product", true,
-        "Enabling strict large query checks disallows the following:\n" +
+        "Enabling strict Cartesian join checks disallows the following:\n" +
         "  Cartesian product (cross join)."),
+    HIVE_STRICT_CHECKS_BUCKETING("hive.strict.checks.bucketing", true,
+        "Enabling strict bucketing checks disallows the following:\n" +
+        "  Load into bucketed tables."),
+
     @Deprecated
     HIVEMAPREDMODE("hive.mapred.mode", null,
         "Deprecated; use hive.strict.checks.* settings instead."),
@@ -4336,11 +4340,14 @@ public class HiveConf extends Configuration {
         "Unsafe compares between different types", ConfVars.HIVE_STRICT_CHECKS_TYPE_SAFETY);
     private static final String NO_CARTESIAN_MSG = makeMessage(
         "Cartesian products", ConfVars.HIVE_STRICT_CHECKS_CARTESIAN);
+    private static final String NO_BUCKETING_MSG = makeMessage(
+        "Load into bucketed tables", ConfVars.HIVE_STRICT_CHECKS_BUCKETING);
 
     private static String makeMessage(String what, ConfVars setting) {
-      return what + " are disabled for safety reasons. If you know what you are doing, please make"
-          + " sure that " + setting.varname + " is set to false and that "
-          + ConfVars.HIVEMAPREDMODE.varname + " is not set to 'strict' to enable them.";
+      return what + " are disabled for safety reasons. If you know what you are doing, please set"
+          + setting.varname + " to false and that " + ConfVars.HIVEMAPREDMODE.varname + " is not"
+          + " set to 'strict' to proceed. Note that if you may get errors or incorrect results if"
+          + " you make a mistake while using some of the unsafe features.";
     }
 
     public static String checkNoLimit(Configuration conf) {
@@ -4360,6 +4367,10 @@ public class HiveConf extends Configuration {
       return isAllowed(conf, ConfVars.HIVE_STRICT_CHECKS_CARTESIAN) ? null : NO_CARTESIAN_MSG;
     }
 
+    public static String checkBucketing(Configuration conf) {
+      return isAllowed(conf, ConfVars.HIVE_STRICT_CHECKS_BUCKETING) ? null : NO_BUCKETING_MSG;
+    }
+
     private static boolean isAllowed(Configuration conf, ConfVars setting) {
       String mode = HiveConf.getVar(conf, ConfVars.HIVEMAPREDMODE, (String)null);
       return (mode != null) ? !"strict".equals(mode) : !HiveConf.getBoolVar(conf, setting);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/data/scripts/q_test_init.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_init.sql b/data/scripts/q_test_init.sql
index 0c8668c..37a6d95 100644
--- a/data/scripts/q_test_init.sql
+++ b/data/scripts/q_test_init.sql
@@ -70,14 +70,18 @@ ANALYZE TABLE src_thrift COMPUTE STATISTICS;
 --
 -- Table srcbucket
 --
+DROP TABLE IF EXISTS srcbucket_tmp;
 DROP TABLE IF EXISTS srcbucket;
 
+CREATE TABLE srcbucket_tmp (key INT, value STRING) STORED AS TEXTFILE;
 CREATE TABLE srcbucket (key INT, value STRING)
 CLUSTERED BY (key) INTO 2 BUCKETS
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket0.txt" INTO TABLE srcbucket;
-LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket1.txt" INTO TABLE srcbucket;
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket0.txt" INTO TABLE srcbucket_tmp;
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket1.txt" INTO TABLE srcbucket_tmp;
+INSERT INTO srcbucket SELECT * FROM srcbucket_tmp;
+DROP TABLE srcbucket_tmp;
  
 ANALYZE TABLE srcbucket COMPUTE STATISTICS;
 
@@ -86,16 +90,20 @@ ANALYZE TABLE srcbucket COMPUTE STATISTICS FOR COLUMNS key,value;
 --
 -- Table srcbucket2
 --
+DROP TABLE IF EXISTS srcbucket_tmp;
 DROP TABLE IF EXISTS srcbucket2;
 
+CREATE TABLE srcbucket_tmp (key INT, value STRING);
 CREATE TABLE srcbucket2 (key INT, value STRING)
 CLUSTERED BY (key) INTO 4 BUCKETS
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket20.txt" INTO TABLE srcbucket2;
-LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket21.txt" INTO TABLE srcbucket2;
-LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket22.txt" INTO TABLE srcbucket2;
-LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket23.txt" INTO TABLE srcbucket2;
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket20.txt" INTO TABLE srcbucket_tmp;
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket21.txt" INTO TABLE srcbucket_tmp;
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket22.txt" INTO TABLE srcbucket_tmp;
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket23.txt" INTO TABLE srcbucket_tmp;
+INSERT INTO srcbucket2 SELECT * FROM srcbucket_tmp;
+DROP TABLE srcbucket_tmp;
 
 ANALYZE TABLE srcbucket2 COMPUTE STATISTICS;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/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 6726d44..bda94ff 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
@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+
 import java.io.IOException;
 import java.io.Serializable;
 import java.net.URI;
@@ -215,6 +219,12 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
         && (ts.partSpec == null || ts.partSpec.size() == 0)) {
       throw new SemanticException(ErrorMsg.NEED_PARTITION_ERROR.getMsg());
     }
+    List<String> bucketCols = ts.tableHandle.getBucketCols();
+    if (bucketCols != null && !bucketCols.isEmpty()) {
+      String error = StrictChecks.checkBucketing(conf);
+      if (error != null) throw new SemanticException("Please load into an intermediate table"
+          + " and use 'insert... select' to allow Hive to enforce bucketing. " + error);
+    }
 
     // make sure the arguments make sense
     List<FileStatus> files = applyConstraintsAndGetFiles(fromURI, fromTree, isLocal);

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q b/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q
index 4193315..6de8c7f 100644
--- a/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q
+++ b/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.exec.concatenate.check.index=true;
 create table src_rc_concatenate_test(key int, value string) stored as rcfile;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/alter_view_failure6.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_view_failure6.q b/ql/src/test/queries/clientnegative/alter_view_failure6.q
index dab7b14..e390409 100644
--- a/ql/src/test/queries/clientnegative/alter_view_failure6.q
+++ b/ql/src/test/queries/clientnegative/alter_view_failure6.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 DROP VIEW xxx7;
 CREATE VIEW xxx7
 PARTITIONED ON (key)

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q b/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q
index 7dce81f..ff982c6 100644
--- a/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q
+++ b/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) 
   partitioned by (ds string) CLUSTERED BY (key) INTO 3 BUCKETS

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q b/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q
index 802fcd9..fa91552 100644
--- a/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q
+++ b/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 -- Although the user has specified a bucketed map-join, the number of buckets in the table
 -- do not match the number of files
 drop table table1;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q b/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q
index ac5abeb..b392c42 100644
--- a/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q
+++ b/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 -- Although the user has specified a bucketed map-join, the number of buckets in the table
 -- do not match the number of files
 drop table table1;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/compare_double_bigint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/compare_double_bigint.q b/ql/src/test/queries/clientnegative/compare_double_bigint.q
index f85c526..6580d88 100644
--- a/ql/src/test/queries/clientnegative/compare_double_bigint.q
+++ b/ql/src/test/queries/clientnegative/compare_double_bigint.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 set hive.mapred.mode=strict;
 
 -- This should fail until we fix the issue with precision when casting a bigint to a double

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/compare_string_bigint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/compare_string_bigint.q b/ql/src/test/queries/clientnegative/compare_string_bigint.q
index 810f65d..ed81637 100644
--- a/ql/src/test/queries/clientnegative/compare_string_bigint.q
+++ b/ql/src/test/queries/clientnegative/compare_string_bigint.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 set hive.mapred.mode=strict;
 
 --This should fail until we fix the issue with precision when casting a bigint to a double

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q b/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q
index 8623db8..e6b7a9f 100644
--- a/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q
+++ b/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 SET hive.metastore.partition.name.whitelist.pattern=[^9]*;
 set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifyTableDirectoryIsEmptyHook;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q b/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q
index 990a686..1d9ae35 100644
--- a/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q
+++ b/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 set hive.test.mode=true;
 set hive.test.mode.prefix=;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/input4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/input4.q b/ql/src/test/queries/clientnegative/input4.q
index 60aea32..03cf127 100644
--- a/ql/src/test/queries/clientnegative/input4.q
+++ b/ql/src/test/queries/clientnegative/input4.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 set hive.mapred.mode=strict;
 
 select * from srcpart a join

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/input_part0_neg.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/input_part0_neg.q b/ql/src/test/queries/clientnegative/input_part0_neg.q
index 4656693..14f947c 100644
--- a/ql/src/test/queries/clientnegative/input_part0_neg.q
+++ b/ql/src/test/queries/clientnegative/input_part0_neg.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 set hive.mapred.mode=strict;
 
 SELECT x.* FROM SRCPART x WHERE key = '2008-04-08';

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q b/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q
index a8f77c2..173dbd0 100644
--- a/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q
+++ b/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.exec.dynamic.partition=true;
 
 create table srcpart_dp like srcpart;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/strict_join.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/strict_join.q b/ql/src/test/queries/clientnegative/strict_join.q
index d618ee2..48b4a60 100644
--- a/ql/src/test/queries/clientnegative/strict_join.q
+++ b/ql/src/test/queries/clientnegative/strict_join.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 set hive.mapred.mode=strict;
 
 SELECT *  FROM src src1 JOIN src src2;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/strict_orderby.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/strict_orderby.q b/ql/src/test/queries/clientnegative/strict_orderby.q
index 781cdbb..928b067 100644
--- a/ql/src/test/queries/clientnegative/strict_orderby.q
+++ b/ql/src/test/queries/clientnegative/strict_orderby.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 set hive.mapred.mode=strict;
 
 EXPLAIN

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientnegative/strict_pruning.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/strict_pruning.q b/ql/src/test/queries/clientnegative/strict_pruning.q
index 270ab2f..834ee82 100644
--- a/ql/src/test/queries/clientnegative/strict_pruning.q
+++ b/ql/src/test/queries/clientnegative/strict_pruning.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false; 
+
 set hive.mapred.mode=strict;
 
 EXPLAIN

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q b/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q
index d2bc440..3a9e14c 100644
--- a/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q
+++ b/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.exec.concatenate.check.index =false;
 create table src_rc_concatenate_test(key int, value string) stored as rcfile;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/alter_merge.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/alter_merge.q b/ql/src/test/queries/clientpositive/alter_merge.q
index 0ff0294..1839005 100644
--- a/ql/src/test/queries/clientpositive/alter_merge.q
+++ b/ql/src/test/queries/clientpositive/alter_merge.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 create table src_rc_merge_test(key int, value string) stored as rcfile;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/alter_merge_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/alter_merge_2.q b/ql/src/test/queries/clientpositive/alter_merge_2.q
index e09703d..b0dd56c 100644
--- a/ql/src/test/queries/clientpositive/alter_merge_2.q
+++ b/ql/src/test/queries/clientpositive/alter_merge_2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 create table src_rc_merge_test_part(key int, value string) partitioned by (ds string, ts string) stored as rcfile;
 
 alter table src_rc_merge_test_part add partition (ds='2012-01-03', ts='2012-01-03+14:46:31');

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/alter_merge_stats.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/alter_merge_stats.q b/ql/src/test/queries/clientpositive/alter_merge_stats.q
index 0af87e2..fedf1a1 100644
--- a/ql/src/test/queries/clientpositive/alter_merge_stats.q
+++ b/ql/src/test/queries/clientpositive/alter_merge_stats.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 create table src_rc_merge_test_stat(key int, value string) stored as rcfile;
 
 load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_stat;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/autoColumnStats_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/autoColumnStats_1.q b/ql/src/test/queries/clientpositive/autoColumnStats_1.q
index bb7252a..7955b07 100644
--- a/ql/src/test/queries/clientpositive/autoColumnStats_1.q
+++ b/ql/src/test/queries/clientpositive/autoColumnStats_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.stats.column.autogather=true;
 set hive.stats.fetch.column.stats=true;
 set hive.exec.dynamic.partition=true;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/autoColumnStats_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/autoColumnStats_2.q b/ql/src/test/queries/clientpositive/autoColumnStats_2.q
index c1abcb1..de86036 100644
--- a/ql/src/test/queries/clientpositive/autoColumnStats_2.q
+++ b/ql/src/test/queries/clientpositive/autoColumnStats_2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.stats.column.autogather=true;
 set hive.stats.fetch.column.stats=true;
 set hive.exec.dynamic.partition=true;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q
index 6c2fbba..a1d5249 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 1 part, 2 bucket & big 2 part, 4 bucket
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q
index aa345b5..b1c60f2 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 1 part, 2 bucket & big 2 part, 4 bucket
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q
index b344bd0..b512cc5 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 1 part, 2 bucket & big 2 part, 4 bucket
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q
index 83b67f8..12ab1fa 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.auto.convert.join=true;
 
 set hive.exec.dynamic.partition.mode=nonstrict;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q
index 2cf809b..e77d937 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 1 part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q
index b14b32b..183f033 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 2 part, 2 bucket & big 1 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q
index b08649e..21f273a 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 2 part, 4 bucket & big 1 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q
index 489f556..8ca04d6 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small no part, 4 bucket & big no part, 2 bucket
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q
index 161bc8a..cf12331 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 2 part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
index cc2ba28..5ec4e26 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.exec.submitviachild=false;
 set hive.exec.submit.local.task.via.child=false;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucket_map_join_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_1.q b/ql/src/test/queries/clientpositive/bucket_map_join_1.q
index deae460..a92ffa4 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_1.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 drop table table1;
 drop table table2;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucket_map_join_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_2.q b/ql/src/test/queries/clientpositive/bucket_map_join_2.q
index f416706..4aa6f57 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_2.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 drop table table1;
 drop table table2;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucket_map_join_spark1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_spark1.q b/ql/src/test/queries/clientpositive/bucket_map_join_spark1.q
index 3f9ccc4..9018b4e 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_spark1.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_spark1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucket_map_join_spark2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_spark2.q b/ql/src/test/queries/clientpositive/bucket_map_join_spark2.q
index 4a70671..6f54737 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_spark2.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_spark2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucket_map_join_spark3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_spark3.q b/ql/src/test/queries/clientpositive/bucket_map_join_spark3.q
index 9ae7986..26a8d48 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_spark3.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_spark3.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q b/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
index ade04dd..aeb244a 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.auto.convert.join=true;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q b/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q
index fd997df..37989ec 100644
--- a/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q
+++ b/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.auto.convert.join=true;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketcontext_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketcontext_1.q b/ql/src/test/queries/clientpositive/bucketcontext_1.q
index ad0f473..d663da4 100644
--- a/ql/src/test/queries/clientpositive/bucketcontext_1.q
+++ b/ql/src/test/queries/clientpositive/bucketcontext_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 1 part, 2 bucket & big 2 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketcontext_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketcontext_2.q b/ql/src/test/queries/clientpositive/bucketcontext_2.q
index 931f55a..cf79a17 100644
--- a/ql/src/test/queries/clientpositive/bucketcontext_2.q
+++ b/ql/src/test/queries/clientpositive/bucketcontext_2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 1 part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketcontext_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketcontext_3.q b/ql/src/test/queries/clientpositive/bucketcontext_3.q
index e03b285..983ddbe 100644
--- a/ql/src/test/queries/clientpositive/bucketcontext_3.q
+++ b/ql/src/test/queries/clientpositive/bucketcontext_3.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 2 part, 2 bucket & big 1 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketcontext_4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketcontext_4.q b/ql/src/test/queries/clientpositive/bucketcontext_4.q
index af08f07..83a9d46 100644
--- a/ql/src/test/queries/clientpositive/bucketcontext_4.q
+++ b/ql/src/test/queries/clientpositive/bucketcontext_4.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 2 part, 4 bucket & big 1 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketcontext_5.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketcontext_5.q b/ql/src/test/queries/clientpositive/bucketcontext_5.q
index 5078072..4c753b9 100644
--- a/ql/src/test/queries/clientpositive/bucketcontext_5.q
+++ b/ql/src/test/queries/clientpositive/bucketcontext_5.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 -- small no part, 4 bucket & big no part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketcontext_6.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketcontext_6.q b/ql/src/test/queries/clientpositive/bucketcontext_6.q
index 5549f3b..a224dac 100644
--- a/ql/src/test/queries/clientpositive/bucketcontext_6.q
+++ b/ql/src/test/queries/clientpositive/bucketcontext_6.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small no part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketcontext_7.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketcontext_7.q b/ql/src/test/queries/clientpositive/bucketcontext_7.q
index 44cdb5e..ef3a51a 100644
--- a/ql/src/test/queries/clientpositive/bucketcontext_7.q
+++ b/ql/src/test/queries/clientpositive/bucketcontext_7.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 2 part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketcontext_8.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketcontext_8.q b/ql/src/test/queries/clientpositive/bucketcontext_8.q
index bb655e2..3c7b168 100644
--- a/ql/src/test/queries/clientpositive/bucketcontext_8.q
+++ b/ql/src/test/queries/clientpositive/bucketcontext_8.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- small 2 part, 2 bucket & big 2 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q b/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q
index a2b0bbe..8516909 100644
--- a/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q
+++ b/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin1.q b/ql/src/test/queries/clientpositive/bucketmapjoin1.q
index 2fd3804..ee27181 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin1.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin10.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin10.q b/ql/src/test/queries/clientpositive/bucketmapjoin10.q
index 09c0ae2..d9b7edf 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin10.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin10.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin11.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin11.q b/ql/src/test/queries/clientpositive/bucketmapjoin11.q
index d330b77..3c630ef 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin11.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin11.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin12.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin12.q b/ql/src/test/queries/clientpositive/bucketmapjoin12.q
index 43a9de4..4229c5c 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin12.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin12.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin2.q b/ql/src/test/queries/clientpositive/bucketmapjoin2.q
index d14261f..3a9816b 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin2.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin3.q b/ql/src/test/queries/clientpositive/bucketmapjoin3.q
index 78c23d5..495d3a4 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin3.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin3.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
 load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin4.q b/ql/src/test/queries/clientpositive/bucketmapjoin4.q
index 54626e7..ab9080c 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin4.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin4.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin5.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin5.q b/ql/src/test/queries/clientpositive/bucketmapjoin5.q
index 6c03bbb..385087e 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin5.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin5.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin7.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin7.q b/ql/src/test/queries/clientpositive/bucketmapjoin7.q
index cd4efe7..3830e9d 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin7.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin7.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin8.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin8.q b/ql/src/test/queries/clientpositive/bucketmapjoin8.q
index f467ea6..99eeabd 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin8.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin8.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin9.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin9.q b/ql/src/test/queries/clientpositive/bucketmapjoin9.q
index f1d5f58..13189bc 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin9.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin9.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q b/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q
index ea140dd..0d84fdd 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q b/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q
index 36d34dc..93780dc 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q b/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q
index 6398fff..41569a2 100644
--- a/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q
+++ b/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 drop table test1;
 drop table test2;
 drop table test3;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/cbo_rp_udaf_percentile_approx_23.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_udaf_percentile_approx_23.q b/ql/src/test/queries/clientpositive/cbo_rp_udaf_percentile_approx_23.q
index 0f2d0dd..c7e6755 100644
--- a/ql/src/test/queries/clientpositive/cbo_rp_udaf_percentile_approx_23.q
+++ b/ql/src/test/queries/clientpositive/cbo_rp_udaf_percentile_approx_23.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.cbo.returnpath.hiveop=true;
 -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/char_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/char_1.q b/ql/src/test/queries/clientpositive/char_1.q
index e617a49..09987cb 100644
--- a/ql/src/test/queries/clientpositive/char_1.q
+++ b/ql/src/test/queries/clientpositive/char_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 -- SORT_QUERY_RESULTS
 
 drop table char1;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/char_serde.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/char_serde.q b/ql/src/test/queries/clientpositive/char_serde.q
index 4340b4d..3959ba5 100644
--- a/ql/src/test/queries/clientpositive/char_serde.q
+++ b/ql/src/test/queries/clientpositive/char_serde.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 drop table if exists char_serde_regex;
 drop table if exists char_serde_lb;
 drop table if exists char_serde_ls;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/create_merge_compressed.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/create_merge_compressed.q b/ql/src/test/queries/clientpositive/create_merge_compressed.q
index 483931b..995401b 100644
--- a/ql/src/test/queries/clientpositive/create_merge_compressed.q
+++ b/ql/src/test/queries/clientpositive/create_merge_compressed.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 create table src_rc_merge_test(key int, value string) stored as rcfile;
 
 load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/explainanalyze_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/explainanalyze_2.q b/ql/src/test/queries/clientpositive/explainanalyze_2.q
index 8b3df87..9cfcf0d 100644
--- a/ql/src/test/queries/clientpositive/explainanalyze_2.q
+++ b/ql/src/test/queries/clientpositive/explainanalyze_2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.explain.user=true;
 
 explain analyze

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/explainanalyze_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/explainanalyze_3.q b/ql/src/test/queries/clientpositive/explainanalyze_3.q
index 61f68a1..6ccdffb 100644
--- a/ql/src/test/queries/clientpositive/explainanalyze_3.q
+++ b/ql/src/test/queries/clientpositive/explainanalyze_3.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider;
 set hive.metastore.filter.hook=org.apache.hadoop.hive.metastore.DefaultMetaStoreFilterHookImpl;
 set hive.mapred.mode=nonstrict;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/explainuser_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/explainuser_1.q b/ql/src/test/queries/clientpositive/explainuser_1.q
index 3791662..a6fbb54 100644
--- a/ql/src/test/queries/clientpositive/explainuser_1.q
+++ b/ql/src/test/queries/clientpositive/explainuser_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=true;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/explainuser_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/explainuser_2.q b/ql/src/test/queries/clientpositive/explainuser_2.q
index 3f8bdb4..ef0e31b 100644
--- a/ql/src/test/queries/clientpositive/explainuser_2.q
+++ b/ql/src/test/queries/clientpositive/explainuser_2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.explain.user=true;
 set hive.metastore.aggregate.stats.cache.enabled=false;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/explainuser_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/explainuser_3.q b/ql/src/test/queries/clientpositive/explainuser_3.q
index 57029f1..282629e 100644
--- a/ql/src/test/queries/clientpositive/explainuser_3.q
+++ b/ql/src/test/queries/clientpositive/explainuser_3.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider;
 set hive.metastore.filter.hook=org.apache.hadoop.hive.metastore.DefaultMetaStoreFilterHookImpl;
 set hive.mapred.mode=nonstrict;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/global_limit.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/global_limit.q b/ql/src/test/queries/clientpositive/global_limit.q
index 8c5f2ea..eefaffd 100644
--- a/ql/src/test/queries/clientpositive/global_limit.q
+++ b/ql/src/test/queries/clientpositive/global_limit.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.limit.optimize.enable=true;
 set hive.limit.optimize.limit.file=2;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q b/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
index 4937be4..5b5bf8e 100644
--- a/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
+++ b/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.exec.infer.bucket.sort=true;
 set hive.exec.infer.bucket.sort.num.buckets.power.two=true;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/insert_orig_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/insert_orig_table.q b/ql/src/test/queries/clientpositive/insert_orig_table.q
index 01fee4e..de408d9 100644
--- a/ql/src/test/queries/clientpositive/insert_orig_table.q
+++ b/ql/src/test/queries/clientpositive/insert_orig_table.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/insert_values_orig_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/insert_values_orig_table.q b/ql/src/test/queries/clientpositive/insert_values_orig_table.q
index e7ae7c6..db4fc82 100644
--- a/ql/src/test/queries/clientpositive/insert_values_orig_table.q
+++ b/ql/src/test/queries/clientpositive/insert_values_orig_table.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/insert_values_orig_table_use_metadata.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/insert_values_orig_table_use_metadata.q b/ql/src/test/queries/clientpositive/insert_values_orig_table_use_metadata.q
index 5e208c4..49c5b0a 100644
--- a/ql/src/test/queries/clientpositive/insert_values_orig_table_use_metadata.q
+++ b/ql/src/test/queries/clientpositive/insert_values_orig_table_use_metadata.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/merge_dynamic_partition.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/merge_dynamic_partition.q b/ql/src/test/queries/clientpositive/merge_dynamic_partition.q
index fa4d442..f70d1c2 100644
--- a/ql/src/test/queries/clientpositive/merge_dynamic_partition.q
+++ b/ql/src/test/queries/clientpositive/merge_dynamic_partition.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q b/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q
index 73a71e6..e61b615 100644
--- a/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q
+++ b/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q b/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q
index 79ae2b9..8caf933 100644
--- a/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q
+++ b/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q b/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q
index 33388c6..9be7f16 100644
--- a/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q
+++ b/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- this test verifies that the block merge task that can follow a query to generate dynamic
 -- partitions does not produce incorrect results by dropping partitions

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q b/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q
index 9a35101..89a8849 100644
--- a/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q
+++ b/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- this is to test the case where some dynamic partitions are merged and some are moved
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/mergejoin.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/mergejoin.q b/ql/src/test/queries/clientpositive/mergejoin.q
index fa53a76..eecd105 100644
--- a/ql/src/test/queries/clientpositive/mergejoin.q
+++ b/ql/src/test/queries/clientpositive/mergejoin.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.join.emit.interval=100000;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/smb_mapjoin_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_1.q b/ql/src/test/queries/clientpositive/smb_mapjoin_1.q
index 4f913e7..6051e4e 100644
--- a/ql/src/test/queries/clientpositive/smb_mapjoin_1.q
+++ b/ql/src/test/queries/clientpositive/smb_mapjoin_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/smb_mapjoin_10.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_10.q b/ql/src/test/queries/clientpositive/smb_mapjoin_10.q
index 1fbe209..cd332c8 100644
--- a/ql/src/test/queries/clientpositive/smb_mapjoin_10.q
+++ b/ql/src/test/queries/clientpositive/smb_mapjoin_10.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 
 create table tmp_smb_bucket_10(userid int, pageid int, postid int, type string) partitioned by (ds string) CLUSTERED BY (userid) SORTED BY (pageid, postid, type, userid) INTO 2 BUCKETS STORED AS RCFILE; 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/smb_mapjoin_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_2.q b/ql/src/test/queries/clientpositive/smb_mapjoin_2.q
index 5e729d9..672cd54 100644
--- a/ql/src/test/queries/clientpositive/smb_mapjoin_2.q
+++ b/ql/src/test/queries/clientpositive/smb_mapjoin_2.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/smb_mapjoin_25.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_25.q b/ql/src/test/queries/clientpositive/smb_mapjoin_25.q
index 834d44c..44da19b 100644
--- a/ql/src/test/queries/clientpositive/smb_mapjoin_25.q
+++ b/ql/src/test/queries/clientpositive/smb_mapjoin_25.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/smb_mapjoin_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_3.q b/ql/src/test/queries/clientpositive/smb_mapjoin_3.q
index 549eb85..48f6ed2 100644
--- a/ql/src/test/queries/clientpositive/smb_mapjoin_3.q
+++ b/ql/src/test/queries/clientpositive/smb_mapjoin_3.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 -- SORT_QUERY_RESULTS
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/smb_mapjoin_4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_4.q b/ql/src/test/queries/clientpositive/smb_mapjoin_4.q
index e9d28ba..4e3fcae 100644
--- a/ql/src/test/queries/clientpositive/smb_mapjoin_4.q
+++ b/ql/src/test/queries/clientpositive/smb_mapjoin_4.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/smb_mapjoin_5.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_5.q b/ql/src/test/queries/clientpositive/smb_mapjoin_5.q
index 5b6eeb8..0decdaf 100644
--- a/ql/src/test/queries/clientpositive/smb_mapjoin_5.q
+++ b/ql/src/test/queries/clientpositive/smb_mapjoin_5.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/smb_mapjoin_7.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_7.q b/ql/src/test/queries/clientpositive/smb_mapjoin_7.q
index bd859b7..f0b41a4 100644
--- a/ql/src/test/queries/clientpositive/smb_mapjoin_7.q
+++ b/ql/src/test/queries/clientpositive/smb_mapjoin_7.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.exec.reducers.max = 1;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/smb_mapjoin_8.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_mapjoin_8.q b/ql/src/test/queries/clientpositive/smb_mapjoin_8.q
index dc6a35f..d2624bb 100644
--- a/ql/src/test/queries/clientpositive/smb_mapjoin_8.q
+++ b/ql/src/test/queries/clientpositive/smb_mapjoin_8.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 ;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/stats1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/stats1.q b/ql/src/test/queries/clientpositive/stats1.q
index e5133c8..a759122 100644
--- a/ql/src/test/queries/clientpositive/stats1.q
+++ b/ql/src/test/queries/clientpositive/stats1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set datanucleus.cache.collections=false;
 set hive.stats.autogather=true;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/stats11.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/stats11.q b/ql/src/test/queries/clientpositive/stats11.q
index d037c00..6cf6d54 100644
--- a/ql/src/test/queries/clientpositive/stats11.q
+++ b/ql/src/test/queries/clientpositive/stats11.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set datanucleus.cache.collections=false;
 set hive.stats.autogather=true;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/stats18.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/stats18.q b/ql/src/test/queries/clientpositive/stats18.q
index e773cd7..6652e0e 100644
--- a/ql/src/test/queries/clientpositive/stats18.q
+++ b/ql/src/test/queries/clientpositive/stats18.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set datanucleus.cache.collections=false;
 set hive.stats.autogather=true;
 set hive.merge.mapfiles=false;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/tez_fsstat.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tez_fsstat.q b/ql/src/test/queries/clientpositive/tez_fsstat.q
index 4aecd71..22e46fe 100644
--- a/ql/src/test/queries/clientpositive/tez_fsstat.q
+++ b/ql/src/test/queries/clientpositive/tez_fsstat.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
 CREATE TABLE t1 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/tez_smb_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tez_smb_1.q b/ql/src/test/queries/clientpositive/tez_smb_1.q
index 089ffe3..6862a9a 100644
--- a/ql/src/test/queries/clientpositive/tez_smb_1.q
+++ b/ql/src/test/queries/clientpositive/tez_smb_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.auto.convert.join=true;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/tez_smb_empty.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tez_smb_empty.q b/ql/src/test/queries/clientpositive/tez_smb_empty.q
index 4661ada..8b3211d 100644
--- a/ql/src/test/queries/clientpositive/tez_smb_empty.q
+++ b/ql/src/test/queries/clientpositive/tez_smb_empty.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.mapjoin.hybridgrace.hashtable=false;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/tez_smb_main.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tez_smb_main.q b/ql/src/test/queries/clientpositive/tez_smb_main.q
index 9ed5959..ee24691 100644
--- a/ql/src/test/queries/clientpositive/tez_smb_main.q
+++ b/ql/src/test/queries/clientpositive/tez_smb_main.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.join.emit.interval=2;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q b/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q
index 7aae8ae..30052eb 100644
--- a/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q
+++ b/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 -- INCLUDE_HADOOP_MAJOR_VERSIONS( 0.20S)
 
 CREATE TABLE bucket (key double, value string) CLUSTERED BY (key) SORTED BY (key DESC)  INTO 4 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q b/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q
index aa4a4ce..80ea489 100644
--- a/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q
+++ b/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
 -- 0.23 changed input order of data in reducer task, which affects result of percentile_approx

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/varchar_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/varchar_1.q b/ql/src/test/queries/clientpositive/varchar_1.q
index efaf3e5..8f397fb 100644
--- a/ql/src/test/queries/clientpositive/varchar_1.q
+++ b/ql/src/test/queries/clientpositive/varchar_1.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 set hive.mapred.mode=nonstrict;
 drop table varchar1;
 drop table varchar1_1;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/queries/clientpositive/varchar_serde.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/varchar_serde.q b/ql/src/test/queries/clientpositive/varchar_serde.q
index ea2a022..7b3d7a9 100644
--- a/ql/src/test/queries/clientpositive/varchar_serde.q
+++ b/ql/src/test/queries/clientpositive/varchar_serde.q
@@ -1,3 +1,5 @@
+set hive.strict.checks.bucketing=false;
+
 drop table if exists varchar_serde_regex;
 drop table if exists varchar_serde_lb;
 drop table if exists varchar_serde_ls;

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientnegative/alter_view_failure6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_failure6.q.out b/ql/src/test/results/clientnegative/alter_view_failure6.q.out
index 70d7622..78416ad 100644
--- a/ql/src/test/results/clientnegative/alter_view_failure6.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_failure6.q.out
@@ -18,5 +18,5 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@xxx7
-FAILED: SemanticException Queries against partitioned tables without a partition filter are disabled for safety reasons. If you know what you are doing, please make sure that hive.strict.checks.large.query is set to false and that hive.mapred.mode is not set to 'strict' to enable them. No partition predicate for Alias "default.xxx7:srcpart" Table "srcpart"
+FAILED: SemanticException Queries against partitioned tables without a partition filter are disabled for safety reasons. If you know what you are doing, please sethive.strict.checks.large.query to false and that hive.mapred.mode is not set to 'strict' to proceed. Note that if you may get errors or incorrect results if you make a mistake while using some of the unsafe features. No partition predicate for Alias "default.xxx7:srcpart" Table "srcpart"
 FAILED: SemanticException [Error 10056]: The query does not reference any valid partition. To run this query, set hive.mapred.mode=nonstrict

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientnegative/compare_double_bigint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/compare_double_bigint.q.out b/ql/src/test/results/clientnegative/compare_double_bigint.q.out
index 10a5667..26a64fc 100644
--- a/ql/src/test/results/clientnegative/compare_double_bigint.q.out
+++ b/ql/src/test/results/clientnegative/compare_double_bigint.q.out
@@ -1 +1 @@
-FAILED: SemanticException Line 0:-1 Wrong arguments '1.0': Unsafe compares between different types are disabled for safety reasons. If you know what you are doing, please make sure that hive.strict.checks.type.safety is set to false and that hive.mapred.mode is not set to 'strict' to enable them.
+FAILED: SemanticException Line 0:-1 Wrong arguments '1.0': Unsafe compares between different types are disabled for safety reasons. If you know what you are doing, please sethive.strict.checks.type.safety to false and that hive.mapred.mode is not set to 'strict' to proceed. Note that if you may get errors or incorrect results if you make a mistake while using some of the unsafe features.

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientnegative/compare_string_bigint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/compare_string_bigint.q.out b/ql/src/test/results/clientnegative/compare_string_bigint.q.out
index c8eb290..d80b314 100644
--- a/ql/src/test/results/clientnegative/compare_string_bigint.q.out
+++ b/ql/src/test/results/clientnegative/compare_string_bigint.q.out
@@ -1 +1 @@
-FAILED: SemanticException Line 0:-1 Wrong arguments ''1'': Unsafe compares between different types are disabled for safety reasons. If you know what you are doing, please make sure that hive.strict.checks.type.safety is set to false and that hive.mapred.mode is not set to 'strict' to enable them.
+FAILED: SemanticException Line 0:-1 Wrong arguments ''1'': Unsafe compares between different types are disabled for safety reasons. If you know what you are doing, please sethive.strict.checks.type.safety to false and that hive.mapred.mode is not set to 'strict' to proceed. Note that if you may get errors or incorrect results if you make a mistake while using some of the unsafe features.

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientnegative/input4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/input4.q.out b/ql/src/test/results/clientnegative/input4.q.out
index 887865d..0e04710 100644
--- a/ql/src/test/results/clientnegative/input4.q.out
+++ b/ql/src/test/results/clientnegative/input4.q.out
@@ -1 +1 @@
-FAILED: SemanticException Cartesian products are disabled for safety reasons. If you know what you are doing, please make sure that hive.strict.checks.cartesian.product is set to false and that hive.mapred.mode is not set to 'strict' to enable them.
+FAILED: SemanticException Cartesian products are disabled for safety reasons. If you know what you are doing, please sethive.strict.checks.cartesian.product to false and that hive.mapred.mode is not set to 'strict' to proceed. Note that if you may get errors or incorrect results if you make a mistake while using some of the unsafe features.

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientnegative/input_part0_neg.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/input_part0_neg.q.out b/ql/src/test/results/clientnegative/input_part0_neg.q.out
index 3abe4b7..8c5761b 100644
--- a/ql/src/test/results/clientnegative/input_part0_neg.q.out
+++ b/ql/src/test/results/clientnegative/input_part0_neg.q.out
@@ -1 +1 @@
-FAILED: SemanticException Queries against partitioned tables without a partition filter are disabled for safety reasons. If you know what you are doing, please make sure that hive.strict.checks.large.query is set to false and that hive.mapred.mode is not set to 'strict' to enable them. No partition predicate for Alias "x" Table "srcpart"
+FAILED: SemanticException Queries against partitioned tables without a partition filter are disabled for safety reasons. If you know what you are doing, please sethive.strict.checks.large.query to false and that hive.mapred.mode is not set to 'strict' to proceed. Note that if you may get errors or incorrect results if you make a mistake while using some of the unsafe features. No partition predicate for Alias "x" Table "srcpart"

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientnegative/strict_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/strict_join.q.out b/ql/src/test/results/clientnegative/strict_join.q.out
index 887865d..0e04710 100644
--- a/ql/src/test/results/clientnegative/strict_join.q.out
+++ b/ql/src/test/results/clientnegative/strict_join.q.out
@@ -1 +1 @@
-FAILED: SemanticException Cartesian products are disabled for safety reasons. If you know what you are doing, please make sure that hive.strict.checks.cartesian.product is set to false and that hive.mapred.mode is not set to 'strict' to enable them.
+FAILED: SemanticException Cartesian products are disabled for safety reasons. If you know what you are doing, please sethive.strict.checks.cartesian.product to false and that hive.mapred.mode is not set to 'strict' to proceed. Note that if you may get errors or incorrect results if you make a mistake while using some of the unsafe features.


[35/35] hive git commit: HIVE-14990 : run all tests for MM tables and fix the issues that are found (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14990 : run all tests for MM tables and fix the issues that are found (Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 05879a8ea584916e971b6401230054bcadd11128
Parents: 536e9b0
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Nov 21 18:45:55 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Nov 21 18:45:55 2016 -0800

----------------------------------------------------------------------
 .../results/clientpositive/llap/mm_all.q.out    | 227 +++++++++++++++++--
 1 file changed, 210 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/05879a8e/ql/src/test/results/clientpositive/llap/mm_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mm_all.q.out b/ql/src/test/results/clientpositive/llap/mm_all.q.out
index 1ba6dce..7a11301 100644
--- a/ql/src/test/results/clientpositive/llap/mm_all.q.out
+++ b/ql/src/test/results/clientpositive/llap/mm_all.q.out
@@ -51,9 +51,9 @@ POSTHOOK: query: create table part_mm(key int) partitioned by (key_mm int) store
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@part_mm
-PREHOOK: query: explain insert into table part_mm partition(key_mm='455') select key from intermediate
+PREHOOK: query: explain insert into table part_mm partition(key_mm=455) select key from intermediate
 PREHOOK: type: QUERY
-POSTHOOK: query: explain insert into table part_mm partition(key_mm='455') select key from intermediate
+POSTHOOK: query: explain insert into table part_mm partition(key_mm=455) select key from intermediate
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
@@ -105,14 +105,14 @@ STAGE PLANS:
   Stage: Stage-3
     Stats-Aggr Operator
 
-PREHOOK: query: insert into table part_mm partition(key_mm='455') select key from intermediate
+PREHOOK: query: insert into table part_mm partition(key_mm=455) select key from intermediate
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
 PREHOOK: Input: default@intermediate@p=455
 PREHOOK: Input: default@intermediate@p=456
 PREHOOK: Input: default@intermediate@p=457
 PREHOOK: Output: default@part_mm@key_mm=455
-POSTHOOK: query: insert into table part_mm partition(key_mm='455') select key from intermediate
+POSTHOOK: query: insert into table part_mm partition(key_mm=455) select key from intermediate
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
@@ -120,14 +120,14 @@ POSTHOOK: Input: default@intermediate@p=456
 POSTHOOK: Input: default@intermediate@p=457
 POSTHOOK: Output: default@part_mm@key_mm=455
 POSTHOOK: Lineage: part_mm PARTITION(key_mm=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: insert into table part_mm partition(key_mm='456') select key from intermediate
+PREHOOK: query: insert into table part_mm partition(key_mm=456) select key from intermediate
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
 PREHOOK: Input: default@intermediate@p=455
 PREHOOK: Input: default@intermediate@p=456
 PREHOOK: Input: default@intermediate@p=457
 PREHOOK: Output: default@part_mm@key_mm=456
-POSTHOOK: query: insert into table part_mm partition(key_mm='456') select key from intermediate
+POSTHOOK: query: insert into table part_mm partition(key_mm=456) select key from intermediate
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
@@ -135,14 +135,14 @@ POSTHOOK: Input: default@intermediate@p=456
 POSTHOOK: Input: default@intermediate@p=457
 POSTHOOK: Output: default@part_mm@key_mm=456
 POSTHOOK: Lineage: part_mm PARTITION(key_mm=456).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: insert into table part_mm partition(key_mm='455') select key from intermediate
+PREHOOK: query: insert into table part_mm partition(key_mm=455) select key from intermediate
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
 PREHOOK: Input: default@intermediate@p=455
 PREHOOK: Input: default@intermediate@p=456
 PREHOOK: Input: default@intermediate@p=457
 PREHOOK: Output: default@part_mm@key_mm=455
-POSTHOOK: query: insert into table part_mm partition(key_mm='455') select key from intermediate
+POSTHOOK: query: insert into table part_mm partition(key_mm=455) select key from intermediate
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
@@ -180,29 +180,37 @@ POSTHOOK: Input: default@part_mm@key_mm=456
 103	455
 103	455
 103	456
-PREHOOK: query: truncate table part_mm partition(key_mm='455')
-PREHOOK: type: TRUNCATETABLE
-PREHOOK: Output: default@part_mm@key_mm=455
-POSTHOOK: query: truncate table part_mm partition(key_mm='455')
-POSTHOOK: type: TRUNCATETABLE
-POSTHOOK: Output: default@part_mm@key_mm=455
-PREHOOK: query: select * from part_mm order by key, key_mm
+PREHOOK: query: -- TODO: doesn't work truncate table part_mm partition(key_mm=455);
+select * from part_mm order by key, key_mm
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part_mm
 PREHOOK: Input: default@part_mm@key_mm=455
 PREHOOK: Input: default@part_mm@key_mm=456
 #### A masked pattern was here ####
-POSTHOOK: query: select * from part_mm order by key, key_mm
+POSTHOOK: query: -- TODO: doesn't work truncate table part_mm partition(key_mm=455);
+select * from part_mm order by key, key_mm
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part_mm
 POSTHOOK: Input: default@part_mm@key_mm=455
 POSTHOOK: Input: default@part_mm@key_mm=456
 #### A masked pattern was here ####
+0	455
+0	455
 0	456
+10	455
+10	455
 10	456
+97	455
+97	455
 97	456
+98	455
+98	455
 98	456
+100	455
+100	455
 100	456
+103	455
+103	455
 103	456
 PREHOOK: query: truncate table part_mm
 PREHOOK: type: TRUNCATETABLE
@@ -2890,8 +2898,8 @@ POSTHOOK: Input: default@intermediate@p=456
 POSTHOOK: Input: default@intermediate@p=457
 POSTHOOK: Output: default@multi1_mm@p=1
 POSTHOOK: Lineage: multi1_mm PARTITION(p=1).key SIMPLE [(intermediate)intermediate.FieldSchema(name:p, type:int, comment:null), ]
-POSTHOOK: Lineage: multi1_mm PARTITION(p=1).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 POSTHOOK: Lineage: multi1_mm PARTITION(p=1).key2 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: multi1_mm PARTITION(p=1).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 POSTHOOK: Lineage: multi1_mm PARTITION(p=1).key2 SIMPLE [(intermediate)intermediate.FieldSchema(name:p, type:int, comment:null), ]
 PREHOOK: query: select key, key2, p from multi1_mm order by key, key2, p
 PREHOOK: type: QUERY
@@ -2967,6 +2975,187 @@ POSTHOOK: query: drop table multi1_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@multi1_mm
 POSTHOOK: Output: default@multi1_mm
+PREHOOK: query: drop table stats_mm
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table stats_mm
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table stats_mm(key int)  tblproperties("transactional"="true", "transactional_properties"="insert_only")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@stats_mm
+POSTHOOK: query: create table stats_mm(key int)  tblproperties("transactional"="true", "transactional_properties"="insert_only")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@stats_mm
+PREHOOK: query: insert overwrite table stats_mm  select key from intermediate
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Input: default@intermediate@p=457
+PREHOOK: Output: default@stats_mm
+POSTHOOK: query: insert overwrite table stats_mm  select key from intermediate
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Input: default@intermediate@p=457
+POSTHOOK: Output: default@stats_mm
+POSTHOOK: Lineage: stats_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: desc formatted stats_mm
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@stats_mm
+POSTHOOK: query: desc formatted stats_mm
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@stats_mm
+# col_name            	data_type           	comment             
+	 	 
+key                 	int                 	                    
+	 	 
+# 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	{\"BASIC_STATS\":\"true\"}
+	numFiles            	3                   
+	numRows             	6                   
+	rawDataSize         	13                  
+	totalSize           	19                  
+	transactional       	true                
+	transactional_properties	insert_only         
+#### 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: insert into table stats_mm  select key from intermediate
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Input: default@intermediate@p=457
+PREHOOK: Output: default@stats_mm
+POSTHOOK: query: insert into table stats_mm  select key from intermediate
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Input: default@intermediate@p=457
+POSTHOOK: Output: default@stats_mm
+POSTHOOK: Lineage: stats_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: desc formatted stats_mm
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@stats_mm
+POSTHOOK: query: desc formatted stats_mm
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@stats_mm
+# col_name            	data_type           	comment             
+	 	 
+key                 	int                 	                    
+	 	 
+# 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	{\"BASIC_STATS\":\"true\"}
+	numFiles            	6                   
+	numRows             	12                  
+	rawDataSize         	26                  
+	totalSize           	38                  
+	transactional       	true                
+	transactional_properties	insert_only         
+#### 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_mm
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@stats_mm
+PREHOOK: Output: default@stats_mm
+POSTHOOK: query: drop table stats_mm
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@stats_mm
+POSTHOOK: Output: default@stats_mm
+PREHOOK: query: drop table stats2_mm
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table stats2_mm
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table stats2_mm tblproperties("transactional"="true", "transactional_properties"="insert_only") as select array(key, value) from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@stats2_mm
+POSTHOOK: query: create table stats2_mm tblproperties("transactional"="true", "transactional_properties"="insert_only") as select array(key, value) from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@stats2_mm
+POSTHOOK: Lineage: stats2_mm.c0 EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: desc formatted stats2_mm
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@stats2_mm
+POSTHOOK: query: desc formatted stats2_mm
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@stats2_mm
+# col_name            	data_type           	comment             
+	 	 
+c0                  	array<string>       	                    
+	 	 
+# 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	{\"BASIC_STATS\":\"true\"}
+	numFiles            	43                  
+	numRows             	500                 
+	rawDataSize         	5312                
+	totalSize           	5812                
+	transactional       	true                
+	transactional_properties	insert_only         
+#### 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 stats2_mm
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@stats2_mm
+PREHOOK: Output: default@stats2_mm
+POSTHOOK: query: drop table stats2_mm
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@stats2_mm
+POSTHOOK: Output: default@stats2_mm
 PREHOOK: query: drop table intermediate
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@intermediate
@@ -2975,3 +3164,7 @@ POSTHOOK: query: drop table intermediate
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Output: default@intermediate
+PREHOOK: query: drop table intermediate
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table intermediate
+POSTHOOK: type: DROPTABLE


[30/35] hive git commit: HIVE-15248: Add Apache header license to TestCustomPartitionVertex (Sergio Pena, reviewed by Aihua Xu)

Posted by se...@apache.org.
HIVE-15248: Add Apache header license to TestCustomPartitionVertex (Sergio Pena, reviewed by Aihua Xu)


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

Branch: refs/heads/hive-14535
Commit: d94ebe8bbfa291323ce4f2cff4d832708b4302ff
Parents: 929ebba
Author: Sergio Pena <se...@cloudera.com>
Authored: Mon Nov 21 08:44:21 2016 -0600
Committer: Sergio Pena <se...@cloudera.com>
Committed: Mon Nov 21 08:44:21 2016 -0600

----------------------------------------------------------------------
 .../ql/exec/tez/TestCustomPartitionVertex.java     | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d94ebe8b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestCustomPartitionVertex.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestCustomPartitionVertex.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestCustomPartitionVertex.java
index dbdd955..3c9757c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestCustomPartitionVertex.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestCustomPartitionVertex.java
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hive.ql.exec.tez;
 
 import com.google.common.collect.HashMultimap;


[06/35] hive git commit: HIVE-15226: Add a different masking comment to qtests blobstore output (Sergio Pena, reviewed by Mohit Sabharwal and Sahil Takiar)

Posted by se...@apache.org.
HIVE-15226: Add a different masking comment to qtests blobstore output (Sergio Pena, reviewed by Mohit Sabharwal and Sahil Takiar)


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

Branch: refs/heads/hive-14535
Commit: c5d147b68a922a69aef163a2287bc3761b3a7186
Parents: 2de8b31
Author: Sergio Pena <se...@cloudera.com>
Authored: Thu Nov 17 12:43:26 2016 -0600
Committer: Sergio Pena <se...@cloudera.com>
Committed: Thu Nov 17 12:44:33 2016 -0600

----------------------------------------------------------------------
 .../results/clientpositive/insert_into.q.out    |  4 ++--
 .../control/AbstractCoreBlobstoreCliDriver.java |  2 ++
 .../org/apache/hadoop/hive/ql/QTestUtil.java    | 24 ++++++++++++++++++--
 3 files changed, 26 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c5d147b6/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out
----------------------------------------------------------------------
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out b/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out
index 919f3e7..93f545d 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out
@@ -4,12 +4,12 @@ POSTHOOK: query: DROP TABLE qtest
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
-PREHOOK: Input: #### A masked pattern was here ####
+PREHOOK: Input: ### test.blobstore.path ###/qtest
 PREHOOK: Output: database:default
 PREHOOK: Output: default@qtest
 #### A masked pattern was here ####
 POSTHOOK: type: CREATETABLE
-POSTHOOK: Input: #### A masked pattern was here ####
+POSTHOOK: Input: ### test.blobstore.path ###/qtest
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@qtest
 PREHOOK: query: INSERT INTO qtest VALUES (1), (10), (100), (1000)

http://git-wip-us.apache.org/repos/asf/hive/blob/c5d147b6/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
index e797eb5..2f57683 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/AbstractCoreBlobstoreCliDriver.java
@@ -163,5 +163,7 @@ public abstract class AbstractCoreBlobstoreCliDriver extends CliAdapter {
     String uid = new SimpleDateFormat("yyyyMMdd.HHmmss.SSS").format(Calendar.getInstance().getTime())
         + "-" + String.format("%03d", (int)(Math.random() * 999));
     testBlobstorePathUnique = testBlobstorePath + uid;
+
+    qt.addPatternWithMaskComment(testBlobstorePathUnique, String.format("### %s ###", HCONF_TEST_BLOBSTORE_PATH));
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c5d147b6/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index ab21589..d08ecc7 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -67,6 +67,8 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileStatus;
@@ -1643,6 +1645,18 @@ public class QTestUtil {
       }
 
       if (!partialMaskWasMatched) {
+        for (Pair<Pattern, String> pair : patternsWithMaskComments) {
+          Pattern pattern = pair.getLeft();
+          String maskComment = pair.getRight();
+
+          matcher = pattern.matcher(line);
+          if (matcher.find()) {
+            line = matcher.replaceAll(maskComment);
+            partialMaskWasMatched = true;
+            break;
+          }
+        }
+
         for (Pattern pattern : patterns) {
           line = pattern.matcher(line).replaceAll(maskPattern);
         }
@@ -1707,8 +1721,7 @@ public class QTestUtil {
       ".*at com\\.sun\\.proxy.*",
       ".*at com\\.jolbox.*",
       ".*at com\\.zaxxer.*",
-      "org\\.apache\\.hadoop\\.hive\\.metastore\\.model\\.MConstraint@([0-9]|[a-z])*",
-      "(s3.?|swift|wasb.?):\\/\\/[\\w\\.\\/-]*"
+      "org\\.apache\\.hadoop\\.hive\\.metastore\\.model\\.MConstraint@([0-9]|[a-z])*"
   });
 
   private final Pattern[] partialReservedPlanMask = toPattern(new String[] {
@@ -1716,6 +1729,13 @@ public class QTestUtil {
       //TODO: add more expected test result here
   });
 
+  /* This list may be modified by specific cli drivers to mask strings that change on every test */
+  private final List<Pair<Pattern, String>> patternsWithMaskComments = new ArrayList<>();
+
+  public void addPatternWithMaskComment(String patternStr, String maskComment) {
+    patternsWithMaskComments.add(ImmutablePair.of(Pattern.compile(patternStr), maskComment));
+  }
+
   public int checkCliDriverResults(String tname) throws Exception {
     assert(qMap.containsKey(tname));
 


[28/35] hive git commit: HIVE-15246: Add a making comment to blobstore staging paths on qtest output (Sergio Pena, reviewed by Mohit Sabharwal and Sahil Takiar)

Posted by se...@apache.org.
HIVE-15246: Add a making comment to blobstore staging paths on qtest output (Sergio Pena, reviewed by Mohit Sabharwal and Sahil Takiar)


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

Branch: refs/heads/hive-14535
Commit: d2cb3277f2fbb71c9ce27034e2538459a31a6e9a
Parents: e2a6273
Author: Sergio Pena <se...@cloudera.com>
Authored: Sun Nov 20 18:03:12 2016 -0600
Committer: Sergio Pena <se...@cloudera.com>
Committed: Sun Nov 20 18:03:12 2016 -0600

----------------------------------------------------------------------
 .../test/queries/clientpositive/insert_into.q   |   3 +
 .../results/clientpositive/insert_into.q.out    | 295 +++++++++++++++++++
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |  10 +-
 3 files changed, 306 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d2cb3277/itests/hive-blobstore/src/test/queries/clientpositive/insert_into.q
----------------------------------------------------------------------
diff --git a/itests/hive-blobstore/src/test/queries/clientpositive/insert_into.q b/itests/hive-blobstore/src/test/queries/clientpositive/insert_into.q
index e36ef1d..919ff7d 100644
--- a/itests/hive-blobstore/src/test/queries/clientpositive/insert_into.q
+++ b/itests/hive-blobstore/src/test/queries/clientpositive/insert_into.q
@@ -1,4 +1,7 @@
+set hive.blobstore.use.blobstore.as.scratchdir=true;
+
 DROP TABLE qtest;
 CREATE TABLE qtest (value int) LOCATION '${hiveconf:test.blobstore.path.unique}/qtest/';
 INSERT INTO qtest VALUES (1), (10), (100), (1000);
+EXPLAIN EXTENDED INSERT INTO qtest VALUES (1), (10), (100), (1000);
 SELECT * FROM qtest;

http://git-wip-us.apache.org/repos/asf/hive/blob/d2cb3277/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out
----------------------------------------------------------------------
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out b/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out
index 93f545d..c25d0c4 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/insert_into.q.out
@@ -21,6 +21,301 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@values__tmp__table__1
 POSTHOOK: Output: default@qtest
 POSTHOOK: Lineage: qtest.value EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: EXPLAIN EXTENDED INSERT INTO qtest VALUES (1), (10), (100), (1000)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN EXTENDED INSERT INTO qtest VALUES (1), (10), (100), (1000)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
+  Stage-4
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
+  Stage-2 depends on stages: Stage-0
+  Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: values__tmp__table__2
+            Statistics: Num rows: 1 Data size: 14 Basic stats: COMPLETE Column stats: NONE
+            GatherStats: false
+            Select Operator
+              expressions: UDFToInteger(tmp_values_col1) (type: int)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 14 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                GlobalTableId: 1
+                directory: ### BLOBSTORE_STAGING_PATH ###
+                NumFilesPerFileSink: 1
+                Statistics: Num rows: 1 Data size: 14 Basic stats: COMPLETE Column stats: NONE
+                Stats Publishing Key Prefix: ### BLOBSTORE_STAGING_PATH ###
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    properties:
+                      bucket_count -1
+                      columns value
+                      columns.comments 
+                      columns.types int
+#### A masked pattern was here ####
+                      location ### test.blobstore.path ###/qtest
+                      name default.qtest
+                      numFiles 1
+                      serialization.ddl struct qtest { i32 value}
+                      serialization.format 1
+                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      totalSize 14
+#### A masked pattern was here ####
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.qtest
+                TotalFiles: 1
+                GatherStats: true
+                MultiFileSpray: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: Values__Tmp__Table__2
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns tmp_values_col1
+              columns.comments 
+              columns.types string
+#### A masked pattern was here ####
+              name default.values__tmp__table__2
+              serialization.ddl struct values__tmp__table__2 { string tmp_values_col1}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                columns tmp_values_col1
+                columns.comments 
+                columns.types string
+#### A masked pattern was here ####
+                name default.values__tmp__table__2
+                serialization.ddl struct values__tmp__table__2 { string tmp_values_col1}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.values__tmp__table__2
+            name: default.values__tmp__table__2
+      Truncated Path -> Alias:
+#### A masked pattern was here ####
+
+  Stage: Stage-7
+    Conditional Operator
+
+  Stage: Stage-4
+    Move Operator
+      files:
+          hdfs directory: true
+          source: ### BLOBSTORE_STAGING_PATH ###
+          destination: ### BLOBSTORE_STAGING_PATH ###
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          source: ### BLOBSTORE_STAGING_PATH ###
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                columns value
+                columns.comments 
+                columns.types int
+#### A masked pattern was here ####
+                location ### test.blobstore.path ###/qtest
+                name default.qtest
+                numFiles 1
+                serialization.ddl struct qtest { i32 value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 14
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.qtest
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+      Stats Aggregation Key Prefix: ### BLOBSTORE_STAGING_PATH ###
+
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            GatherStats: false
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              directory: ### BLOBSTORE_STAGING_PATH ###
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns value
+                    columns.comments 
+                    columns.types int
+#### A masked pattern was here ####
+                    location ### test.blobstore.path ###/qtest
+                    name default.qtest
+                    numFiles 1
+                    serialization.ddl struct qtest { i32 value}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 14
+#### A masked pattern was here ####
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.qtest
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Path -> Alias:
+        ### BLOBSTORE_STAGING_PATH ###
+      Path -> Partition:
+        ### BLOBSTORE_STAGING_PATH ###
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns value
+              columns.comments 
+              columns.types int
+#### A masked pattern was here ####
+              location ### test.blobstore.path ###/qtest
+              name default.qtest
+              numFiles 1
+              serialization.ddl struct qtest { i32 value}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 14
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                columns value
+                columns.comments 
+                columns.types int
+#### A masked pattern was here ####
+                location ### test.blobstore.path ###/qtest
+                name default.qtest
+                numFiles 1
+                serialization.ddl struct qtest { i32 value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 14
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.qtest
+            name: default.qtest
+      Truncated Path -> Alias:
+        ### BLOBSTORE_STAGING_PATH ###
+
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            GatherStats: false
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              directory: ### BLOBSTORE_STAGING_PATH ###
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns value
+                    columns.comments 
+                    columns.types int
+#### A masked pattern was here ####
+                    location ### test.blobstore.path ###/qtest
+                    name default.qtest
+                    numFiles 1
+                    serialization.ddl struct qtest { i32 value}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 14
+#### A masked pattern was here ####
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.qtest
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Path -> Alias:
+        ### BLOBSTORE_STAGING_PATH ###
+      Path -> Partition:
+        ### BLOBSTORE_STAGING_PATH ###
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns value
+              columns.comments 
+              columns.types int
+#### A masked pattern was here ####
+              location ### test.blobstore.path ###/qtest
+              name default.qtest
+              numFiles 1
+              serialization.ddl struct qtest { i32 value}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 14
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                columns value
+                columns.comments 
+                columns.types int
+#### A masked pattern was here ####
+                location ### test.blobstore.path ###/qtest
+                name default.qtest
+                numFiles 1
+                serialization.ddl struct qtest { i32 value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 14
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.qtest
+            name: default.qtest
+      Truncated Path -> Alias:
+        ### BLOBSTORE_STAGING_PATH ###
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+          source: ### BLOBSTORE_STAGING_PATH ###
+          destination: ### BLOBSTORE_STAGING_PATH ###
+
 PREHOOK: query: SELECT * FROM qtest
 PREHOOK: type: QUERY
 PREHOOK: Input: default@qtest

http://git-wip-us.apache.org/repos/asf/hive/blob/d2cb3277/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index d08ecc7..5bf23e7 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1730,10 +1730,16 @@ public class QTestUtil {
   });
 
   /* This list may be modified by specific cli drivers to mask strings that change on every test */
-  private final List<Pair<Pattern, String>> patternsWithMaskComments = new ArrayList<>();
+  private List<Pair<Pattern, String>> patternsWithMaskComments = new ArrayList<Pair<Pattern, String>>() {{
+    add(toPatternPair("(s3.?|swift|wasb.?).*hive-staging.*","### BLOBSTORE_STAGING_PATH ###"));
+  }};
+
+  private Pair<Pattern, String> toPatternPair(String patternStr, String maskComment) {
+    return ImmutablePair.of(Pattern.compile(patternStr), maskComment);
+  }
 
   public void addPatternWithMaskComment(String patternStr, String maskComment) {
-    patternsWithMaskComments.add(ImmutablePair.of(Pattern.compile(patternStr), maskComment));
+    patternsWithMaskComments.add(toPatternPair(patternStr, maskComment));
   }
 
   public int checkCliDriverResults(String tname) throws Exception {


[21/35] hive git commit: HIVE-15148 : disallow loading data into bucketed tables (by default) (Sergey Shelukhin, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientnegative/strict_orderby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/strict_orderby.q.out b/ql/src/test/results/clientnegative/strict_orderby.q.out
index 08c1062..3cc2a0c 100644
--- a/ql/src/test/results/clientnegative/strict_orderby.q.out
+++ b/ql/src/test/results/clientnegative/strict_orderby.q.out
@@ -1 +1 @@
-FAILED: SemanticException 4:47 Order by-s without limit are disabled for safety reasons. If you know what you are doing, please make sure that hive.strict.checks.large.query is set to false and that hive.mapred.mode is not set to 'strict' to enable them.. Error encountered near token 'key'
+FAILED: SemanticException 4:47 Order by-s without limit are disabled for safety reasons. If you know what you are doing, please sethive.strict.checks.large.query to false and that hive.mapred.mode is not set to 'strict' to proceed. Note that if you may get errors or incorrect results if you make a mistake while using some of the unsafe features.. Error encountered near token 'key'

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientnegative/strict_pruning.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/strict_pruning.q.out b/ql/src/test/results/clientnegative/strict_pruning.q.out
index d45a2b8..76b9083 100644
--- a/ql/src/test/results/clientnegative/strict_pruning.q.out
+++ b/ql/src/test/results/clientnegative/strict_pruning.q.out
@@ -1 +1 @@
-FAILED: SemanticException Queries against partitioned tables without a partition filter are disabled for safety reasons. If you know what you are doing, please make sure that hive.strict.checks.large.query is set to false and that hive.mapred.mode is not set to 'strict' to enable them. No partition predicate for Alias "srcpart" Table "srcpart"
+FAILED: SemanticException Queries against partitioned tables without a partition filter are disabled for safety reasons. If you know what you are doing, please sethive.strict.checks.large.query to false and that hive.mapred.mode is not set to 'strict' to proceed. Note that if you may get errors or incorrect results if you make a mistake while using some of the unsafe features. No partition predicate for Alias "srcpart" Table "srcpart"

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientpositive/sample2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sample2.q.out b/ql/src/test/results/clientpositive/sample2.q.out
index 096805d..06690df 100644
--- a/ql/src/test/results/clientpositive/sample2.q.out
+++ b/ql/src/test/results/clientpositive/sample2.q.out
@@ -81,7 +81,7 @@ STAGE PLANS:
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket0.txt
+            base file name: 000000_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -126,7 +126,7 @@ STAGE PLANS:
               name: default.srcbucket
             name: default.srcbucket
       Truncated Path -> Alias:
-        /srcbucket/srcbucket0.txt [s]
+        /srcbucket/000000_0 [s]
 
   Stage: Stage-7
     Conditional Operator
@@ -362,496 +362,496 @@ POSTHOOK: query: SELECT dest1.* FROM dest1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@dest1
 #### A masked pattern was here ####
-474	val_475
-62	val_63
-468	val_469
-272	val_273
-448	val_449
-246	val_247
-440	val_441
-278	val_279
-296	val_297
-428	val_429
-126	val_127
-106	val_107
-356	val_357
 490	val_491
-402	val_403
-128	val_129
-10	val_11
-226	val_227
-110	val_111
-0	val_1
-240	val_241
-286	val_287
-408	val_409
-476	val_477
-482	val_483
-48	val_49
-424	val_425
-226	val_227
-494	val_495
-488	val_489
-94	val_95
-50	val_51
-402	val_403
-128	val_129
-468	val_469
-314	val_315
-224	val_225
-344	val_345
-4	val_5
-206	val_207
-114	val_115
-56	val_57
-114	val_115
-254	val_255
-390	val_391
-304	val_305
-264	val_265
-196	val_197
-238	val_239
-20	val_21
-492	val_493
-82	val_83
-58	val_59
-86	val_87
-438	val_439
-360	val_361
-222	val_223
-42	val_43
-338	val_339
-68	val_69
-16	val_17
-492	val_493
-376	val_377
-120	val_121
-306	val_307
-426	val_427
-132	val_133
-446	val_447
-386	val_387
-388	val_389
-184	val_185
-284	val_285
-246	val_247
-262	val_263
-122	val_123
-438	val_439
-390	val_391
-352	val_353
-226	val_227
-328	val_329
-382	val_383
-342	val_343
-480	val_481
-102	val_103
-480	val_481
-318	val_319
-392	val_393
-476	val_477
-258	val_259
-174	val_175
-252	val_253
-114	val_115
-264	val_265
-48	val_49
-336	val_337
-340	val_341
-390	val_391
-484	val_485
-6	val_7
-260	val_261
-2	val_3
-170	val_171
-164	val_165
-118	val_119
-310	val_311
-104	val_105
-80	val_81
-326	val_327
-450	val_451
-140	val_141
-212	val_213
-308	val_309
-30	val_31
-358	val_359
-416	val_417
-42	val_43
-386	val_387
-454	val_455
-364	val_365
-20	val_21
-52	val_53
-40	val_41
-8	val_9
-168	val_169
-384	val_385
-324	val_325
-310	val_311
-206	val_207
-404	val_405
-206	val_207
-226	val_227
-262	val_263
-260	val_261
-328	val_329
-322	val_323
-122	val_123
-404	val_405
-384	val_385
-76	val_77
-116	val_117
-42	val_43
-104	val_105
-406	val_407
-32	val_33
-132	val_133
-192	val_193
-58	val_59
-70	val_71
 356	val_357
-352	val_353
-52	val_53
-330	val_331
-138	val_139
-160	val_161
-454	val_455
-76	val_77
-174	val_175
-412	val_413
-16	val_17
-204	val_205
+106	val_107
 126	val_127
-274	val_275
-374	val_375
-494	val_495
-216	val_217
-470	val_471
-196	val_197
-302	val_303
-450	val_451
-12	val_13
-398	val_399
-334	val_335
-384	val_385
-60	val_61
-442	val_443
-52	val_53
-404	val_405
-446	val_447
-300	val_301
-0	val_1
-268	val_269
-392	val_393
-104	val_105
-436	val_437
-156	val_157
-118	val_119
-172	val_173
-244	val_245
-6	val_7
-284	val_285
-164	val_165
-136	val_137
-462	val_463
-432	val_433
-496	val_497
-144	val_145
-408	val_409
-152	val_153
-382	val_383
-348	val_349
-122	val_123
-292	val_293
-182	val_183
-474	val_475
-310	val_311
-52	val_53
-486	val_487
-152	val_153
-378	val_379
-414	val_415
-256	val_257
-292	val_293
-412	val_413
-40	val_41
-478	val_479
-178	val_179
-100	val_101
-156	val_157
-228	val_229
-22	val_23
-248	val_249
-402	val_403
-62	val_63
-162	val_163
-244	val_245
-276	val_277
-46	val_47
-78	val_79
-134	val_135
-196	val_197
-410	val_411
-82	val_83
+428	val_429
+296	val_297
+278	val_279
 440	val_441
-100	val_101
-308	val_309
-430	val_431
+246	val_247
+448	val_449
+272	val_273
 468	val_469
-152	val_153
-138	val_139
-76	val_77
-300	val_301
-478	val_479
-118	val_119
-178	val_179
-242	val_243
-244	val_245
-238	val_238
-86	val_86
-278	val_278
-98	val_98
-484	val_484
-150	val_150
-224	val_224
-66	val_66
-128	val_128
-146	val_146
-406	val_406
-374	val_374
+62	val_63
+474	val_475
+200	val_200
+400	val_400
+90	val_90
+126	val_126
+222	val_222
+414	val_414
+194	val_194
+348	val_348
 152	val_152
-82	val_82
-166	val_166
-430	val_430
-252	val_252
-292	val_292
-338	val_338
-446	val_446
-394	val_394
-482	val_482
-174	val_174
-494	val_494
-466	val_466
-208	val_208
-174	val_174
-396	val_396
-162	val_162
-266	val_266
-342	val_342
-0	val_0
-128	val_128
-316	val_316
-302	val_302
-438	val_438
-170	val_170
-20	val_20
-378	val_378
-92	val_92
-72	val_72
-4	val_4
-280	val_280
-208	val_208
-356	val_356
-382	val_382
-498	val_498
-386	val_386
-192	val_192
-286	val_286
-176	val_176
-54	val_54
-138	val_138
-216	val_216
-430	val_430
-278	val_278
-176	val_176
-318	val_318
-332	val_332
-180	val_180
-284	val_284
-12	val_12
-230	val_230
-260	val_260
-404	val_404
-384	val_384
-272	val_272
-138	val_138
+448	val_448
+28	val_28
 84	val_84
+344	val_344
+18	val_18
 348	val_348
-466	val_466
-58	val_58
-8	val_8
-230	val_230
-208	val_208
-348	val_348
-24	val_24
-172	val_172
-42	val_42
-158	val_158
-496	val_496
-0	val_0
-322	val_322
-468	val_468
-454	val_454
-100	val_100
-298	val_298
-418	val_418
-96	val_96
-26	val_26
-230	val_230
-120	val_120
-404	val_404
-436	val_436
-156	val_156
-468	val_468
-308	val_308
-196	val_196
-288	val_288
-98	val_98
-282	val_282
-318	val_318
-318	val_318
-470	val_470
-316	val_316
-0	val_0
-490	val_490
-364	val_364
-118	val_118
-134	val_134
-282	val_282
-138	val_138
-238	val_238
-118	val_118
-72	val_72
-90	val_90
-10	val_10
-306	val_306
-224	val_224
-242	val_242
-392	val_392
-272	val_272
-242	val_242
-452	val_452
-226	val_226
-402	val_402
-396	val_396
-58	val_58
-336	val_336
-168	val_168
-34	val_34
-472	val_472
-322	val_322
-498	val_498
-160	val_160
-42	val_42
-430	val_430
+186	val_186
+362	val_362
 458	val_458
-78	val_78
-76	val_76
+146	val_146
+498	val_498
+298	val_298
+100	val_100
 492	val_492
-218	val_218
-228	val_228
-138	val_138
-30	val_30
-64	val_64
-468	val_468
-76	val_76
-74	val_74
-342	val_342
-230	val_230
-368	val_368
-296	val_296
-216	val_216
-344	val_344
-274	val_274
-116	val_116
+462	val_462
+18	val_18
+384	val_384
+134	val_134
+26	val_26
 256	val_256
-70	val_70
+384	val_384
+454	val_454
+406	val_406
+462	val_462
+214	val_214
+172	val_172
+136	val_136
 480	val_480
-288	val_288
-244	val_244
+460	val_460
+310	val_310
+468	val_468
+178	val_178
+478	val_478
+230	val_230
+120	val_120
+444	val_444
+248	val_248
+360	val_360
+200	val_200
+414	val_414
 438	val_438
-128	val_128
-432	val_432
-202	val_202
-316	val_316
-280	val_280
-2	val_2
-80	val_80
-44	val_44
+70	val_70
 104	val_104
-466	val_466
-366	val_366
-406	val_406
-190	val_190
-406	val_406
-114	val_114
-258	val_258
-90	val_90
-262	val_262
-348	val_348
+24	val_24
+480	val_480
+70	val_70
+382	val_382
 424	val_424
-12	val_12
-396	val_396
 164	val_164
-454	val_454
-478	val_478
 298	val_298
+478	val_478
+454	val_454
 164	val_164
+396	val_396
+12	val_12
 424	val_424
-382	val_382
-70	val_70
-480	val_480
-24	val_24
+348	val_348
+262	val_262
+90	val_90
+258	val_258
+114	val_114
+406	val_406
+190	val_190
+406	val_406
+366	val_366
+466	val_466
 104	val_104
-70	val_70
+44	val_44
+80	val_80
+2	val_2
+280	val_280
+316	val_316
+202	val_202
+432	val_432
+128	val_128
 438	val_438
-414	val_414
-200	val_200
-360	val_360
-248	val_248
-444	val_444
-120	val_120
-230	val_230
-478	val_478
-178	val_178
-468	val_468
-310	val_310
-460	val_460
+244	val_244
+288	val_288
 480	val_480
-136	val_136
-172	val_172
-214	val_214
-462	val_462
-406	val_406
-454	val_454
-384	val_384
+70	val_70
 256	val_256
-26	val_26
-134	val_134
-384	val_384
-18	val_18
-462	val_462
+116	val_116
+274	val_274
+344	val_344
+216	val_216
+296	val_296
+368	val_368
+230	val_230
+342	val_342
+74	val_74
+76	val_76
+468	val_468
+64	val_64
+30	val_30
+138	val_138
+228	val_228
+218	val_218
 492	val_492
-100	val_100
-298	val_298
-498	val_498
-146	val_146
+76	val_76
+78	val_78
 458	val_458
-362	val_362
-186	val_186
+430	val_430
+42	val_42
+160	val_160
+498	val_498
+322	val_322
+472	val_472
+34	val_34
+168	val_168
+336	val_336
+58	val_58
+396	val_396
+402	val_402
+226	val_226
+452	val_452
+242	val_242
+272	val_272
+392	val_392
+242	val_242
+224	val_224
+306	val_306
+10	val_10
+90	val_90
+72	val_72
+118	val_118
+238	val_238
+138	val_138
+282	val_282
+134	val_134
+118	val_118
+364	val_364
+490	val_490
+0	val_0
+316	val_316
+470	val_470
+318	val_318
+318	val_318
+282	val_282
+98	val_98
+288	val_288
+196	val_196
+308	val_308
+468	val_468
+156	val_156
+436	val_436
+404	val_404
+120	val_120
+230	val_230
+26	val_26
+96	val_96
+418	val_418
+298	val_298
+100	val_100
+454	val_454
+468	val_468
+322	val_322
+0	val_0
+496	val_496
+158	val_158
+42	val_42
+172	val_172
+24	val_24
+348	val_348
+208	val_208
+230	val_230
+8	val_8
+58	val_58
+466	val_466
 348	val_348
-18	val_18
-344	val_344
 84	val_84
-28	val_28
-448	val_448
+138	val_138
+272	val_272
+384	val_384
+404	val_404
+260	val_260
+230	val_230
+12	val_12
+284	val_284
+180	val_180
+332	val_332
+318	val_318
+176	val_176
+278	val_278
+430	val_430
+216	val_216
+138	val_138
+54	val_54
+176	val_176
+286	val_286
+192	val_192
+386	val_386
+498	val_498
+382	val_382
+356	val_356
+208	val_208
+280	val_280
+4	val_4
+72	val_72
+92	val_92
+378	val_378
+20	val_20
+170	val_170
+438	val_438
+302	val_302
+316	val_316
+128	val_128
+0	val_0
+342	val_342
+266	val_266
+162	val_162
+396	val_396
+174	val_174
+208	val_208
+466	val_466
+494	val_494
+174	val_174
+482	val_482
+394	val_394
+446	val_446
+338	val_338
+292	val_292
+252	val_252
+430	val_430
+166	val_166
+82	val_82
 152	val_152
-348	val_348
-194	val_194
-414	val_414
-222	val_222
-126	val_126
-90	val_90
-400	val_400
-200	val_200
+374	val_374
+406	val_406
+146	val_146
+128	val_128
+66	val_66
+224	val_224
+150	val_150
+484	val_484
+98	val_98
+278	val_278
+86	val_86
+238	val_238
+244	val_245
+242	val_243
+178	val_179
+118	val_119
+478	val_479
+300	val_301
+76	val_77
+138	val_139
+152	val_153
+468	val_469
+430	val_431
+308	val_309
+100	val_101
+440	val_441
+82	val_83
+410	val_411
+196	val_197
+134	val_135
+78	val_79
+46	val_47
+276	val_277
+244	val_245
+162	val_163
+62	val_63
+402	val_403
+248	val_249
+22	val_23
+228	val_229
+156	val_157
+100	val_101
+178	val_179
+478	val_479
+40	val_41
+412	val_413
+292	val_293
+256	val_257
+414	val_415
+378	val_379
+152	val_153
+486	val_487
+52	val_53
+310	val_311
+474	val_475
+182	val_183
+292	val_293
+122	val_123
+348	val_349
+382	val_383
+152	val_153
+408	val_409
+144	val_145
+496	val_497
+432	val_433
+462	val_463
+136	val_137
+164	val_165
+284	val_285
+6	val_7
+244	val_245
+172	val_173
+118	val_119
+156	val_157
+436	val_437
+104	val_105
+392	val_393
+268	val_269
+0	val_1
+300	val_301
+446	val_447
+404	val_405
+52	val_53
+442	val_443
+60	val_61
+384	val_385
+334	val_335
+398	val_399
+12	val_13
+450	val_451
+302	val_303
+196	val_197
+470	val_471
+216	val_217
+494	val_495
+374	val_375
+274	val_275
+126	val_127
+204	val_205
+16	val_17
+412	val_413
+174	val_175
+76	val_77
+454	val_455
+160	val_161
+138	val_139
+330	val_331
+52	val_53
+352	val_353
+356	val_357
+70	val_71
+58	val_59
+192	val_193
+132	val_133
+32	val_33
+406	val_407
+104	val_105
+42	val_43
+116	val_117
+76	val_77
+384	val_385
+404	val_405
+122	val_123
+322	val_323
+328	val_329
+260	val_261
+262	val_263
+226	val_227
+206	val_207
+404	val_405
+206	val_207
+310	val_311
+324	val_325
+384	val_385
+168	val_169
+8	val_9
+40	val_41
+52	val_53
+20	val_21
+364	val_365
+454	val_455
+386	val_387
+42	val_43
+416	val_417
+358	val_359
+30	val_31
+308	val_309
+212	val_213
+140	val_141
+450	val_451
+326	val_327
+80	val_81
+104	val_105
+310	val_311
+118	val_119
+164	val_165
+170	val_171
+2	val_3
+260	val_261
+6	val_7
+484	val_485
+390	val_391
+340	val_341
+336	val_337
+48	val_49
+264	val_265
+114	val_115
+252	val_253
+174	val_175
+258	val_259
+476	val_477
+392	val_393
+318	val_319
+480	val_481
+102	val_103
+480	val_481
+342	val_343
+382	val_383
+328	val_329
+226	val_227
+352	val_353
+390	val_391
+438	val_439
+122	val_123
+262	val_263
+246	val_247
+284	val_285
+184	val_185
+388	val_389
+386	val_387
+446	val_447
+132	val_133
+426	val_427
+306	val_307
+120	val_121
+376	val_377
+492	val_493
+16	val_17
+68	val_69
+338	val_339
+42	val_43
+222	val_223
+360	val_361
+438	val_439
+86	val_87
+58	val_59
+82	val_83
+492	val_493
+20	val_21
+238	val_239
+196	val_197
+264	val_265
+304	val_305
+390	val_391
+254	val_255
+114	val_115
+56	val_57
+114	val_115
+206	val_207
+4	val_5
+344	val_345
+224	val_225
+314	val_315
+468	val_469
+128	val_129
+402	val_403
+50	val_51
+94	val_95
+488	val_489
+494	val_495
+226	val_227
+424	val_425
+48	val_49
+482	val_483
+476	val_477
+408	val_409
+286	val_287
+240	val_241
+0	val_1
+110	val_111
+226	val_227
+10	val_11
+128	val_129
+402	val_403

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientpositive/sample4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sample4.q.out b/ql/src/test/results/clientpositive/sample4.q.out
index 72395c9..4652dd8 100644
--- a/ql/src/test/results/clientpositive/sample4.q.out
+++ b/ql/src/test/results/clientpositive/sample4.q.out
@@ -81,7 +81,7 @@ STAGE PLANS:
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket0.txt
+            base file name: 000000_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -126,7 +126,7 @@ STAGE PLANS:
               name: default.srcbucket
             name: default.srcbucket
       Truncated Path -> Alias:
-        /srcbucket/srcbucket0.txt [s]
+        /srcbucket/000000_0 [s]
 
   Stage: Stage-7
     Conditional Operator
@@ -362,496 +362,496 @@ POSTHOOK: query: SELECT dest1.* FROM dest1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@dest1
 #### A masked pattern was here ####
-474	val_475
-62	val_63
-468	val_469
-272	val_273
-448	val_449
-246	val_247
-440	val_441
-278	val_279
-296	val_297
-428	val_429
-126	val_127
-106	val_107
-356	val_357
 490	val_491
-402	val_403
-128	val_129
-10	val_11
-226	val_227
-110	val_111
-0	val_1
-240	val_241
-286	val_287
-408	val_409
-476	val_477
-482	val_483
-48	val_49
-424	val_425
-226	val_227
-494	val_495
-488	val_489
-94	val_95
-50	val_51
-402	val_403
-128	val_129
-468	val_469
-314	val_315
-224	val_225
-344	val_345
-4	val_5
-206	val_207
-114	val_115
-56	val_57
-114	val_115
-254	val_255
-390	val_391
-304	val_305
-264	val_265
-196	val_197
-238	val_239
-20	val_21
-492	val_493
-82	val_83
-58	val_59
-86	val_87
-438	val_439
-360	val_361
-222	val_223
-42	val_43
-338	val_339
-68	val_69
-16	val_17
-492	val_493
-376	val_377
-120	val_121
-306	val_307
-426	val_427
-132	val_133
-446	val_447
-386	val_387
-388	val_389
-184	val_185
-284	val_285
-246	val_247
-262	val_263
-122	val_123
-438	val_439
-390	val_391
-352	val_353
-226	val_227
-328	val_329
-382	val_383
-342	val_343
-480	val_481
-102	val_103
-480	val_481
-318	val_319
-392	val_393
-476	val_477
-258	val_259
-174	val_175
-252	val_253
-114	val_115
-264	val_265
-48	val_49
-336	val_337
-340	val_341
-390	val_391
-484	val_485
-6	val_7
-260	val_261
-2	val_3
-170	val_171
-164	val_165
-118	val_119
-310	val_311
-104	val_105
-80	val_81
-326	val_327
-450	val_451
-140	val_141
-212	val_213
-308	val_309
-30	val_31
-358	val_359
-416	val_417
-42	val_43
-386	val_387
-454	val_455
-364	val_365
-20	val_21
-52	val_53
-40	val_41
-8	val_9
-168	val_169
-384	val_385
-324	val_325
-310	val_311
-206	val_207
-404	val_405
-206	val_207
-226	val_227
-262	val_263
-260	val_261
-328	val_329
-322	val_323
-122	val_123
-404	val_405
-384	val_385
-76	val_77
-116	val_117
-42	val_43
-104	val_105
-406	val_407
-32	val_33
-132	val_133
-192	val_193
-58	val_59
-70	val_71
 356	val_357
-352	val_353
-52	val_53
-330	val_331
-138	val_139
-160	val_161
-454	val_455
-76	val_77
-174	val_175
-412	val_413
-16	val_17
-204	val_205
+106	val_107
 126	val_127
-274	val_275
-374	val_375
-494	val_495
-216	val_217
-470	val_471
-196	val_197
-302	val_303
-450	val_451
-12	val_13
-398	val_399
-334	val_335
-384	val_385
-60	val_61
-442	val_443
-52	val_53
-404	val_405
-446	val_447
-300	val_301
-0	val_1
-268	val_269
-392	val_393
-104	val_105
-436	val_437
-156	val_157
-118	val_119
-172	val_173
-244	val_245
-6	val_7
-284	val_285
-164	val_165
-136	val_137
-462	val_463
-432	val_433
-496	val_497
-144	val_145
-408	val_409
-152	val_153
-382	val_383
-348	val_349
-122	val_123
-292	val_293
-182	val_183
-474	val_475
-310	val_311
-52	val_53
-486	val_487
-152	val_153
-378	val_379
-414	val_415
-256	val_257
-292	val_293
-412	val_413
-40	val_41
-478	val_479
-178	val_179
-100	val_101
-156	val_157
-228	val_229
-22	val_23
-248	val_249
-402	val_403
-62	val_63
-162	val_163
-244	val_245
-276	val_277
-46	val_47
-78	val_79
-134	val_135
-196	val_197
-410	val_411
-82	val_83
+428	val_429
+296	val_297
+278	val_279
 440	val_441
-100	val_101
-308	val_309
-430	val_431
+246	val_247
+448	val_449
+272	val_273
 468	val_469
-152	val_153
-138	val_139
-76	val_77
-300	val_301
-478	val_479
-118	val_119
-178	val_179
-242	val_243
-244	val_245
-238	val_238
-86	val_86
-278	val_278
-98	val_98
-484	val_484
-150	val_150
-224	val_224
-66	val_66
-128	val_128
-146	val_146
-406	val_406
-374	val_374
+62	val_63
+474	val_475
+200	val_200
+400	val_400
+90	val_90
+126	val_126
+222	val_222
+414	val_414
+194	val_194
+348	val_348
 152	val_152
-82	val_82
-166	val_166
-430	val_430
-252	val_252
-292	val_292
-338	val_338
-446	val_446
-394	val_394
-482	val_482
-174	val_174
-494	val_494
-466	val_466
-208	val_208
-174	val_174
-396	val_396
-162	val_162
-266	val_266
-342	val_342
-0	val_0
-128	val_128
-316	val_316
-302	val_302
-438	val_438
-170	val_170
-20	val_20
-378	val_378
-92	val_92
-72	val_72
-4	val_4
-280	val_280
-208	val_208
-356	val_356
-382	val_382
-498	val_498
-386	val_386
-192	val_192
-286	val_286
-176	val_176
-54	val_54
-138	val_138
-216	val_216
-430	val_430
-278	val_278
-176	val_176
-318	val_318
-332	val_332
-180	val_180
-284	val_284
-12	val_12
-230	val_230
-260	val_260
-404	val_404
-384	val_384
-272	val_272
-138	val_138
+448	val_448
+28	val_28
 84	val_84
+344	val_344
+18	val_18
 348	val_348
-466	val_466
-58	val_58
-8	val_8
-230	val_230
-208	val_208
-348	val_348
-24	val_24
-172	val_172
-42	val_42
-158	val_158
-496	val_496
-0	val_0
-322	val_322
-468	val_468
-454	val_454
-100	val_100
-298	val_298
-418	val_418
-96	val_96
-26	val_26
-230	val_230
-120	val_120
-404	val_404
-436	val_436
-156	val_156
-468	val_468
-308	val_308
-196	val_196
-288	val_288
-98	val_98
-282	val_282
-318	val_318
-318	val_318
-470	val_470
-316	val_316
-0	val_0
-490	val_490
-364	val_364
-118	val_118
-134	val_134
-282	val_282
-138	val_138
-238	val_238
-118	val_118
-72	val_72
-90	val_90
-10	val_10
-306	val_306
-224	val_224
-242	val_242
-392	val_392
-272	val_272
-242	val_242
-452	val_452
-226	val_226
-402	val_402
-396	val_396
-58	val_58
-336	val_336
-168	val_168
-34	val_34
-472	val_472
-322	val_322
-498	val_498
-160	val_160
-42	val_42
-430	val_430
+186	val_186
+362	val_362
 458	val_458
-78	val_78
-76	val_76
+146	val_146
+498	val_498
+298	val_298
+100	val_100
 492	val_492
-218	val_218
-228	val_228
-138	val_138
-30	val_30
-64	val_64
-468	val_468
-76	val_76
-74	val_74
-342	val_342
-230	val_230
-368	val_368
-296	val_296
-216	val_216
-344	val_344
-274	val_274
-116	val_116
+462	val_462
+18	val_18
+384	val_384
+134	val_134
+26	val_26
 256	val_256
-70	val_70
+384	val_384
+454	val_454
+406	val_406
+462	val_462
+214	val_214
+172	val_172
+136	val_136
 480	val_480
-288	val_288
-244	val_244
+460	val_460
+310	val_310
+468	val_468
+178	val_178
+478	val_478
+230	val_230
+120	val_120
+444	val_444
+248	val_248
+360	val_360
+200	val_200
+414	val_414
 438	val_438
-128	val_128
-432	val_432
-202	val_202
-316	val_316
-280	val_280
-2	val_2
-80	val_80
-44	val_44
+70	val_70
 104	val_104
-466	val_466
-366	val_366
-406	val_406
-190	val_190
-406	val_406
-114	val_114
-258	val_258
-90	val_90
-262	val_262
-348	val_348
+24	val_24
+480	val_480
+70	val_70
+382	val_382
 424	val_424
-12	val_12
-396	val_396
 164	val_164
-454	val_454
-478	val_478
 298	val_298
+478	val_478
+454	val_454
 164	val_164
+396	val_396
+12	val_12
 424	val_424
-382	val_382
-70	val_70
-480	val_480
-24	val_24
+348	val_348
+262	val_262
+90	val_90
+258	val_258
+114	val_114
+406	val_406
+190	val_190
+406	val_406
+366	val_366
+466	val_466
 104	val_104
-70	val_70
+44	val_44
+80	val_80
+2	val_2
+280	val_280
+316	val_316
+202	val_202
+432	val_432
+128	val_128
 438	val_438
-414	val_414
-200	val_200
-360	val_360
-248	val_248
-444	val_444
-120	val_120
-230	val_230
-478	val_478
-178	val_178
-468	val_468
-310	val_310
-460	val_460
+244	val_244
+288	val_288
 480	val_480
-136	val_136
-172	val_172
-214	val_214
-462	val_462
-406	val_406
-454	val_454
-384	val_384
+70	val_70
 256	val_256
-26	val_26
-134	val_134
-384	val_384
-18	val_18
-462	val_462
+116	val_116
+274	val_274
+344	val_344
+216	val_216
+296	val_296
+368	val_368
+230	val_230
+342	val_342
+74	val_74
+76	val_76
+468	val_468
+64	val_64
+30	val_30
+138	val_138
+228	val_228
+218	val_218
 492	val_492
-100	val_100
-298	val_298
-498	val_498
-146	val_146
+76	val_76
+78	val_78
 458	val_458
-362	val_362
-186	val_186
+430	val_430
+42	val_42
+160	val_160
+498	val_498
+322	val_322
+472	val_472
+34	val_34
+168	val_168
+336	val_336
+58	val_58
+396	val_396
+402	val_402
+226	val_226
+452	val_452
+242	val_242
+272	val_272
+392	val_392
+242	val_242
+224	val_224
+306	val_306
+10	val_10
+90	val_90
+72	val_72
+118	val_118
+238	val_238
+138	val_138
+282	val_282
+134	val_134
+118	val_118
+364	val_364
+490	val_490
+0	val_0
+316	val_316
+470	val_470
+318	val_318
+318	val_318
+282	val_282
+98	val_98
+288	val_288
+196	val_196
+308	val_308
+468	val_468
+156	val_156
+436	val_436
+404	val_404
+120	val_120
+230	val_230
+26	val_26
+96	val_96
+418	val_418
+298	val_298
+100	val_100
+454	val_454
+468	val_468
+322	val_322
+0	val_0
+496	val_496
+158	val_158
+42	val_42
+172	val_172
+24	val_24
+348	val_348
+208	val_208
+230	val_230
+8	val_8
+58	val_58
+466	val_466
 348	val_348
-18	val_18
-344	val_344
 84	val_84
-28	val_28
-448	val_448
+138	val_138
+272	val_272
+384	val_384
+404	val_404
+260	val_260
+230	val_230
+12	val_12
+284	val_284
+180	val_180
+332	val_332
+318	val_318
+176	val_176
+278	val_278
+430	val_430
+216	val_216
+138	val_138
+54	val_54
+176	val_176
+286	val_286
+192	val_192
+386	val_386
+498	val_498
+382	val_382
+356	val_356
+208	val_208
+280	val_280
+4	val_4
+72	val_72
+92	val_92
+378	val_378
+20	val_20
+170	val_170
+438	val_438
+302	val_302
+316	val_316
+128	val_128
+0	val_0
+342	val_342
+266	val_266
+162	val_162
+396	val_396
+174	val_174
+208	val_208
+466	val_466
+494	val_494
+174	val_174
+482	val_482
+394	val_394
+446	val_446
+338	val_338
+292	val_292
+252	val_252
+430	val_430
+166	val_166
+82	val_82
 152	val_152
-348	val_348
-194	val_194
-414	val_414
-222	val_222
-126	val_126
-90	val_90
-400	val_400
-200	val_200
+374	val_374
+406	val_406
+146	val_146
+128	val_128
+66	val_66
+224	val_224
+150	val_150
+484	val_484
+98	val_98
+278	val_278
+86	val_86
+238	val_238
+244	val_245
+242	val_243
+178	val_179
+118	val_119
+478	val_479
+300	val_301
+76	val_77
+138	val_139
+152	val_153
+468	val_469
+430	val_431
+308	val_309
+100	val_101
+440	val_441
+82	val_83
+410	val_411
+196	val_197
+134	val_135
+78	val_79
+46	val_47
+276	val_277
+244	val_245
+162	val_163
+62	val_63
+402	val_403
+248	val_249
+22	val_23
+228	val_229
+156	val_157
+100	val_101
+178	val_179
+478	val_479
+40	val_41
+412	val_413
+292	val_293
+256	val_257
+414	val_415
+378	val_379
+152	val_153
+486	val_487
+52	val_53
+310	val_311
+474	val_475
+182	val_183
+292	val_293
+122	val_123
+348	val_349
+382	val_383
+152	val_153
+408	val_409
+144	val_145
+496	val_497
+432	val_433
+462	val_463
+136	val_137
+164	val_165
+284	val_285
+6	val_7
+244	val_245
+172	val_173
+118	val_119
+156	val_157
+436	val_437
+104	val_105
+392	val_393
+268	val_269
+0	val_1
+300	val_301
+446	val_447
+404	val_405
+52	val_53
+442	val_443
+60	val_61
+384	val_385
+334	val_335
+398	val_399
+12	val_13
+450	val_451
+302	val_303
+196	val_197
+470	val_471
+216	val_217
+494	val_495
+374	val_375
+274	val_275
+126	val_127
+204	val_205
+16	val_17
+412	val_413
+174	val_175
+76	val_77
+454	val_455
+160	val_161
+138	val_139
+330	val_331
+52	val_53
+352	val_353
+356	val_357
+70	val_71
+58	val_59
+192	val_193
+132	val_133
+32	val_33
+406	val_407
+104	val_105
+42	val_43
+116	val_117
+76	val_77
+384	val_385
+404	val_405
+122	val_123
+322	val_323
+328	val_329
+260	val_261
+262	val_263
+226	val_227
+206	val_207
+404	val_405
+206	val_207
+310	val_311
+324	val_325
+384	val_385
+168	val_169
+8	val_9
+40	val_41
+52	val_53
+20	val_21
+364	val_365
+454	val_455
+386	val_387
+42	val_43
+416	val_417
+358	val_359
+30	val_31
+308	val_309
+212	val_213
+140	val_141
+450	val_451
+326	val_327
+80	val_81
+104	val_105
+310	val_311
+118	val_119
+164	val_165
+170	val_171
+2	val_3
+260	val_261
+6	val_7
+484	val_485
+390	val_391
+340	val_341
+336	val_337
+48	val_49
+264	val_265
+114	val_115
+252	val_253
+174	val_175
+258	val_259
+476	val_477
+392	val_393
+318	val_319
+480	val_481
+102	val_103
+480	val_481
+342	val_343
+382	val_383
+328	val_329
+226	val_227
+352	val_353
+390	val_391
+438	val_439
+122	val_123
+262	val_263
+246	val_247
+284	val_285
+184	val_185
+388	val_389
+386	val_387
+446	val_447
+132	val_133
+426	val_427
+306	val_307
+120	val_121
+376	val_377
+492	val_493
+16	val_17
+68	val_69
+338	val_339
+42	val_43
+222	val_223
+360	val_361
+438	val_439
+86	val_87
+58	val_59
+82	val_83
+492	val_493
+20	val_21
+238	val_239
+196	val_197
+264	val_265
+304	val_305
+390	val_391
+254	val_255
+114	val_115
+56	val_57
+114	val_115
+206	val_207
+4	val_5
+344	val_345
+224	val_225
+314	val_315
+468	val_469
+128	val_129
+402	val_403
+50	val_51
+94	val_95
+488	val_489
+494	val_495
+226	val_227
+424	val_425
+48	val_49
+482	val_483
+476	val_477
+408	val_409
+286	val_287
+240	val_241
+0	val_1
+110	val_111
+226	val_227
+10	val_11
+128	val_129
+402	val_403

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientpositive/sample6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sample6.q.out b/ql/src/test/results/clientpositive/sample6.q.out
index a34258d..475fe02 100644
--- a/ql/src/test/results/clientpositive/sample6.q.out
+++ b/ql/src/test/results/clientpositive/sample6.q.out
@@ -79,7 +79,7 @@ STAGE PLANS:
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket0.txt
+            base file name: 000000_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -124,7 +124,7 @@ STAGE PLANS:
               name: default.srcbucket
             name: default.srcbucket
       Truncated Path -> Alias:
-        /srcbucket/srcbucket0.txt [s]
+        /srcbucket/000000_0 [s]
 
   Stage: Stage-7
     Conditional Operator
@@ -360,255 +360,255 @@ POSTHOOK: query: SELECT dest1.* FROM dest1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@dest1
 #### A masked pattern was here ####
-468	val_469
-272	val_273
-448	val_449
-440	val_441
-296	val_297
-428	val_429
 356	val_357
-128	val_129
-0	val_1
-240	val_241
-408	val_409
-476	val_477
-48	val_49
-424	val_425
-488	val_489
-128	val_129
-468	val_469
-224	val_225
-344	val_345
-4	val_5
-56	val_57
-304	val_305
-264	val_265
-196	val_197
-20	val_21
-492	val_493
-360	val_361
-68	val_69
-16	val_17
-492	val_493
-376	val_377
-120	val_121
-132	val_133
-388	val_389
-184	val_185
-284	val_285
-352	val_353
-328	val_329
-480	val_481
-480	val_481
-392	val_393
-476	val_477
-252	val_253
-264	val_265
-48	val_49
-336	val_337
-340	val_341
-484	val_485
-260	val_261
-164	val_165
-104	val_105
-80	val_81
-140	val_141
-212	val_213
-308	val_309
-416	val_417
-364	val_365
-20	val_21
-52	val_53
-40	val_41
-8	val_9
-168	val_169
-384	val_385
-324	val_325
-404	val_405
-260	val_261
-328	val_329
-404	val_405
-384	val_385
-76	val_77
-116	val_117
-104	val_105
-32	val_33
-132	val_133
-192	val_193
-356	val_357
-352	val_353
-52	val_53
-160	val_161
-76	val_77
-412	val_413
-16	val_17
-204	val_205
-216	val_217
-196	val_197
-12	val_13
-384	val_385
-60	val_61
-52	val_53
-404	val_405
-300	val_301
-0	val_1
-268	val_269
-392	val_393
-104	val_105
-436	val_437
-156	val_157
-172	val_173
-244	val_245
-284	val_285
-164	val_165
-136	val_137
-432	val_433
-496	val_497
-144	val_145
-408	val_409
-152	val_153
-348	val_349
-292	val_293
-52	val_53
-152	val_153
-256	val_257
-292	val_293
-412	val_413
-40	val_41
-100	val_101
-156	val_157
-228	val_229
-248	val_249
-244	val_245
-276	val_277
-196	val_197
+428	val_429
+296	val_297
 440	val_441
-100	val_101
-308	val_309
+448	val_449
+272	val_273
 468	val_469
-152	val_153
-76	val_77
-300	val_301
-244	val_245
-484	val_484
-224	val_224
-128	val_128
+200	val_200
+400	val_400
+348	val_348
 152	val_152
-252	val_252
-292	val_292
-208	val_208
-396	val_396
-0	val_0
-128	val_128
-316	val_316
-20	val_20
-92	val_92
-72	val_72
-4	val_4
-280	val_280
-208	val_208
-356	val_356
-192	val_192
-176	val_176
-216	val_216
-176	val_176
-332	val_332
-180	val_180
-284	val_284
-12	val_12
-260	val_260
-404	val_404
-384	val_384
-272	val_272
+448	val_448
+28	val_28
 84	val_84
+344	val_344
 348	val_348
-8	val_8
-208	val_208
-348	val_348
-24	val_24
-172	val_172
-496	val_496
-0	val_0
-468	val_468
 100	val_100
-96	val_96
-120	val_120
-404	val_404
-436	val_436
-156	val_156
-468	val_468
-308	val_308
-196	val_196
-288	val_288
-316	val_316
-0	val_0
-364	val_364
-72	val_72
-224	val_224
-392	val_392
-272	val_272
-452	val_452
-396	val_396
-336	val_336
-168	val_168
-472	val_472
-160	val_160
-76	val_76
 492	val_492
-228	val_228
-64	val_64
-468	val_468
-76	val_76
-368	val_368
-296	val_296
-216	val_216
-344	val_344
-116	val_116
+384	val_384
 256	val_256
+384	val_384
+172	val_172
+136	val_136
 480	val_480
-288	val_288
-244	val_244
-128	val_128
-432	val_432
-316	val_316
-280	val_280
-80	val_80
-44	val_44
+460	val_460
+468	val_468
+120	val_120
+444	val_444
+248	val_248
+360	val_360
+200	val_200
 104	val_104
-348	val_348
+24	val_24
+480	val_480
 424	val_424
-12	val_12
-396	val_396
 164	val_164
 164	val_164
+396	val_396
+12	val_12
 424	val_424
-480	val_480
-24	val_24
+348	val_348
 104	val_104
-200	val_200
-360	val_360
-248	val_248
-444	val_444
-120	val_120
-468	val_468
-460	val_460
+44	val_44
+80	val_80
+280	val_280
+316	val_316
+432	val_432
+128	val_128
+244	val_244
+288	val_288
 480	val_480
-136	val_136
-172	val_172
-384	val_384
 256	val_256
-384	val_384
+116	val_116
+344	val_344
+216	val_216
+296	val_296
+368	val_368
+76	val_76
+468	val_468
+64	val_64
+228	val_228
 492	val_492
+76	val_76
+160	val_160
+472	val_472
+168	val_168
+336	val_336
+396	val_396
+452	val_452
+272	val_272
+392	val_392
+224	val_224
+72	val_72
+364	val_364
+0	val_0
+316	val_316
+288	val_288
+196	val_196
+308	val_308
+468	val_468
+156	val_156
+436	val_436
+404	val_404
+120	val_120
+96	val_96
 100	val_100
+468	val_468
+0	val_0
+496	val_496
+172	val_172
+24	val_24
+348	val_348
+208	val_208
+8	val_8
 348	val_348
-344	val_344
 84	val_84
-28	val_28
-448	val_448
+272	val_272
+384	val_384
+404	val_404
+260	val_260
+12	val_12
+284	val_284
+180	val_180
+332	val_332
+176	val_176
+216	val_216
+176	val_176
+192	val_192
+356	val_356
+208	val_208
+280	val_280
+4	val_4
+72	val_72
+92	val_92
+20	val_20
+316	val_316
+128	val_128
+0	val_0
+396	val_396
+208	val_208
+292	val_292
+252	val_252
 152	val_152
-348	val_348
-400	val_400
-200	val_200
+128	val_128
+224	val_224
+484	val_484
+244	val_245
+300	val_301
+76	val_77
+152	val_153
+468	val_469
+308	val_309
+100	val_101
+440	val_441
+196	val_197
+276	val_277
+244	val_245
+248	val_249
+228	val_229
+156	val_157
+100	val_101
+40	val_41
+412	val_413
+292	val_293
+256	val_257
+152	val_153
+52	val_53
+292	val_293
+348	val_349
+152	val_153
+408	val_409
+144	val_145
+496	val_497
+432	val_433
+136	val_137
+164	val_165
+284	val_285
+244	val_245
+172	val_173
+156	val_157
+436	val_437
+104	val_105
+392	val_393
+268	val_269
+0	val_1
+300	val_301
+404	val_405
+52	val_53
+60	val_61
+384	val_385
+12	val_13
+196	val_197
+216	val_217
+204	val_205
+16	val_17
+412	val_413
+76	val_77
+160	val_161
+52	val_53
+352	val_353
+356	val_357
+192	val_193
+132	val_133
+32	val_33
+104	val_105
+116	val_117
+76	val_77
+384	val_385
+404	val_405
+328	val_329
+260	val_261
+404	val_405
+324	val_325
+384	val_385
+168	val_169
+8	val_9
+40	val_41
+52	val_53
+20	val_21
+364	val_365
+416	val_417
+308	val_309
+212	val_213
+140	val_141
+80	val_81
+104	val_105
+164	val_165
+260	val_261
+484	val_485
+340	val_341
+336	val_337
+48	val_49
+264	val_265
+252	val_253
+476	val_477
+392	val_393
+480	val_481
+480	val_481
+328	val_329
+352	val_353
+284	val_285
+184	val_185
+388	val_389
+132	val_133
+120	val_121
+376	val_377
+492	val_493
+16	val_17
+68	val_69
+360	val_361
+492	val_493
+20	val_21
+196	val_197
+264	val_265
+304	val_305
+56	val_57
+4	val_5
+344	val_345
+224	val_225
+468	val_469
+128	val_129
+488	val_489
+424	val_425
+48	val_49
+476	val_477
+408	val_409
+240	val_241
+0	val_1
+128	val_129
 PREHOOK: query: EXPLAIN EXTENDED SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 4 OUT OF 4 on key) s
 ORDER BY key, value
 PREHOOK: type: QUERY
@@ -648,7 +648,7 @@ STAGE PLANS:
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket1.txt
+            base file name: 000001_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -693,7 +693,7 @@ STAGE PLANS:
               name: default.srcbucket
             name: default.srcbucket
       Truncated Path -> Alias:
-        /srcbucket/srcbucket1.txt [s]
+        /srcbucket/000001_0 [s]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
@@ -1017,7 +1017,7 @@ STAGE PLANS:
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket0.txt
+            base file name: 000000_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -1062,7 +1062,7 @@ STAGE PLANS:
               name: default.srcbucket
             name: default.srcbucket
       Truncated Path -> Alias:
-        /srcbucket/srcbucket0.txt [s]
+        /srcbucket/000000_0 [s]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
@@ -2556,7 +2556,7 @@ STAGE PLANS:
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket20.txt
+            base file name: 000000_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -2602,7 +2602,7 @@ STAGE PLANS:
             name: default.srcbucket2
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket22.txt
+            base file name: 000002_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -2647,8 +2647,8 @@ STAGE PLANS:
               name: default.srcbucket2
             name: default.srcbucket2
       Truncated Path -> Alias:
-        /srcbucket2/srcbucket20.txt [s]
-        /srcbucket2/srcbucket22.txt [s]
+        /srcbucket2/000000_0 [s]
+        /srcbucket2/000002_0 [s]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
@@ -2700,26 +2700,62 @@ POSTHOOK: Input: default@srcbucket2
 2	val_2
 4	val_4
 8	val_8
+10	val_10
+12	val_12
+12	val_12
+18	val_18
+18	val_18
 20	val_20
 24	val_24
 24	val_24
 26	val_26
 26	val_26
 28	val_28
+30	val_30
+34	val_34
 42	val_42
 42	val_42
 44	val_44
+54	val_54
+58	val_58
+58	val_58
 64	val_64
 66	val_66
+70	val_70
+70	val_70
+70	val_70
+72	val_72
+72	val_72
+74	val_74
+76	val_76
+76	val_76
+78	val_78
 80	val_80
 82	val_82
 84	val_84
 84	val_84
 86	val_86
+90	val_90
+90	val_90
+90	val_90
+92	val_92
+96	val_96
+98	val_98
+98	val_98
+100	val_100
+100	val_100
+104	val_104
+104	val_104
 114	val_114
 116	val_116
 118	val_118
 118	val_118
+120	val_120
+120	val_120
+126	val_126
+128	val_128
+128	val_128
+128	val_128
 134	val_134
 134	val_134
 136	val_136
@@ -2727,11 +2763,19 @@ POSTHOOK: Input: default@srcbucket2
 138	val_138
 138	val_138
 138	val_138
+146	val_146
+146	val_146
 150	val_150
 152	val_152
 152	val_152
 156	val_156
 158	val_158
+160	val_160
+162	val_162
+164	val_164
+164	val_164
+166	val_166
+168	val_168
 170	val_170
 172	val_172
 172	val_172
@@ -2740,6 +2784,8 @@ POSTHOOK: Input: default@srcbucket2
 176	val_176
 176	val_176
 178	val_178
+180	val_180
+186	val_186
 190	val_190
 192	val_192
 194	val_194
@@ -2750,18 +2796,38 @@ POSTHOOK: Input: default@srcbucket2
 208	val_208
 208	val_208
 208	val_208
+214	val_214
+216	val_216
+216	val_216
+218	val_218
 222	val_222
 224	val_224
 224	val_224
 226	val_226
 228	val_228
+230	val_230
+230	val_230
+230	val_230
+230	val_230
+230	val_230
+238	val_238
+238	val_238
 242	val_242
 242	val_242
 244	val_244
 248	val_248
+252	val_252
+256	val_256
+256	val_256
+258	val_258
 260	val_260
 262	val_262
 266	val_266
+272	val_272
+272	val_272
+274	val_274
+278	val_278
+278	val_278
 280	val_280
 280	val_280
 282	val_282
@@ -2770,6 +2836,14 @@ POSTHOOK: Input: default@srcbucket2
 286	val_286
 288	val_288
 288	val_288
+292	val_292
+296	val_296
+298	val_298
+298	val_298
+298	val_298
+302	val_302
+306	val_306
+308	val_308
 310	val_310
 316	val_316
 316	val_316
@@ -2777,12 +2851,34 @@ POSTHOOK: Input: default@srcbucket2
 318	val_318
 318	val_318
 318	val_318
+322	val_322
+322	val_322
 332	val_332
 336	val_336
 338	val_338
+342	val_342
+342	val_342
+344	val_344
+344	val_344
+348	val_348
+348	val_348
+348	val_348
+348	val_348
+348	val_348
 356	val_356
+360	val_360
+362	val_362
+364	val_364
+366	val_366
+368	val_368
 374	val_374
 378	val_378
+382	val_382
+382	val_382
+384	val_384
+384	val_384
+384	val_384
+386	val_386
 392	val_392
 394	val_394
 396	val_396
@@ -2796,11 +2892,28 @@ POSTHOOK: Input: default@srcbucket2
 406	val_406
 406	val_406
 406	val_406
+414	val_414
+414	val_414
+418	val_418
 424	val_424
 424	val_424
+430	val_430
+430	val_430
+430	val_430
+432	val_432
+436	val_436
+438	val_438
+438	val_438
+438	val_438
 444	val_444
 446	val_446
 448	val_448
+452	val_452
+454	val_454
+454	val_454
+454	val_454
+458	val_458
+458	val_458
 460	val_460
 462	val_462
 462	val_462
@@ -2811,11 +2924,23 @@ POSTHOOK: Input: default@srcbucket2
 468	val_468
 468	val_468
 468	val_468
+470	val_470
+472	val_472
+478	val_478
+478	val_478
 480	val_480
 480	val_480
 480	val_480
 482	val_482
 484	val_484
+490	val_490
+492	val_492
+492	val_492
+494	val_494
+496	val_496
+498	val_498
+498	val_498
+498	val_498
 PREHOOK: query: EXPLAIN EXTENDED SELECT s.* FROM srcbucket2 TABLESAMPLE (BUCKET 2 OUT OF 4 on key) s
 ORDER BY key, value
 PREHOOK: type: QUERY
@@ -2855,7 +2980,7 @@ STAGE PLANS:
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket21.txt
+            base file name: 000001_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -2900,7 +3025,7 @@ STAGE PLANS:
               name: default.srcbucket2
             name: default.srcbucket2
       Truncated Path -> Alias:
-        /srcbucket2/srcbucket21.txt [s]
+        /srcbucket2/000001_0 [s]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
@@ -2950,22 +3075,103 @@ POSTHOOK: Input: default@srcbucket2
 5	val_5
 5	val_5
 9	val_9
+17	val_17
+33	val_33
+37	val_37
+37	val_37
 41	val_41
+53	val_53
+57	val_57
+65	val_65
+69	val_69
+77	val_77
 85	val_85
+97	val_97
+97	val_97
+105	val_105
+113	val_113
+113	val_113
+125	val_125
+125	val_125
+129	val_129
+129	val_129
 133	val_133
 137	val_137
 137	val_137
+145	val_145
+149	val_149
+149	val_149
+153	val_153
+157	val_157
+165	val_165
+165	val_165
+169	val_169
+169	val_169
+169	val_169
+169	val_169
 177	val_177
+181	val_181
+189	val_189
+193	val_193
+193	val_193
+193	val_193
+197	val_197
+197	val_197
+201	val_201
+205	val_205
+205	val_205
+209	val_209
+209	val_209
+213	val_213
+213	val_213
+217	val_217
+217	val_217
 221	val_221
 221	val_221
 229	val_229
 229	val_229
+233	val_233
+233	val_233
+237	val_237
+237	val_237
+241	val_241
+249	val_249
+257	val_257
 265	val_265
 265	val_265
+273	val_273
+273	val_273
+273	val_273
+277	val_277
+277	val_277
+277	val_277
+277	val_277
+281	val_281
+281	val_281
+285	val_285
+289	val_289
+305	val_305
+309	val_309
+309	val_309
 317	val_317
 317	val_317
+321	val_321
+321	val_321
+325	val_325
+325	val_325
+333	val_333
+333	val_333
+341	val_341
+345	val_345
 353	val_353
 353	val_353
+365	val_365
+369	val_369
+369	val_369
+369	val_369
+373	val_373
+377	val_377
+389	val_389
 393	val_393
 397	val_397
 397	val_397
@@ -2977,13 +3183,32 @@ POSTHOOK: Input: default@srcbucket2
 409	val_409
 409	val_409
 409	val_409
+413	val_413
+413	val_413
+417	val_417
+417	val_417
+417	val_417
+421	val_421
+429	val_429
+429	val_429
+437	val_437
 449	val_449
+453	val_453
+457	val_457
+469	val_469
+469	val_469
+469	val_469
+469	val_469
+469	val_469
+477	val_477
 481	val_481
 485	val_485
 489	val_489
 489	val_489
 489	val_489
 489	val_489
+493	val_493
+497	val_497
 PREHOOK: query: CREATE TABLE empty_bucket (key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientpositive/sample7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sample7.q.out b/ql/src/test/results/clientpositive/sample7.q.out
index 51a45dd..1d0f0e8 100644
--- a/ql/src/test/results/clientpositive/sample7.q.out
+++ b/ql/src/test/results/clientpositive/sample7.q.out
@@ -80,7 +80,7 @@ STAGE PLANS:
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket0.txt
+            base file name: 000000_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -125,7 +125,7 @@ STAGE PLANS:
               name: default.srcbucket
             name: default.srcbucket
       Truncated Path -> Alias:
-        /srcbucket/srcbucket0.txt [s]
+        /srcbucket/000000_0 [s]
 
   Stage: Stage-7
     Conditional Operator
@@ -363,201 +363,201 @@ POSTHOOK: query: SELECT dest1.* FROM dest1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@dest1
 #### A masked pattern was here ####
-468	val_469
-272	val_273
-448	val_449
-440	val_441
-296	val_297
-428	val_429
-356	val_357
-128	val_129
-240	val_241
-408	val_409
-476	val_477
-424	val_425
-488	val_489
-128	val_129
-468	val_469
-224	val_225
-344	val_345
-304	val_305
-264	val_265
-196	val_197
-492	val_493
-360	val_361
-492	val_493
-376	val_377
-120	val_121
-132	val_133
-388	val_389
-184	val_185
-284	val_285
-352	val_353
-328	val_329
-480	val_481
-480	val_481
-392	val_393
-476	val_477
-252	val_253
-264	val_265
-336	val_337
-340	val_341
-484	val_485
-260	val_261
-164	val_165
-104	val_105
-140	val_141
-212	val_213
-308	val_309
-416	val_417
-364	val_365
-168	val_169
-384	val_385
-324	val_325
-404	val_405
-260	val_261
-328	val_329
-404	val_405
-384	val_385
-116	val_117
-104	val_105
-132	val_133
-192	val_193
 356	val_357
-352	val_353
-160	val_161
-412	val_413
-204	val_205
-216	val_217
-196	val_197
-384	val_385
-404	val_405
-300	val_301
-268	val_269
-392	val_393
-104	val_105
-436	val_437
-156	val_157
-172	val_173
-244	val_245
-284	val_285
-164	val_165
-136	val_137
-432	val_433
-496	val_497
-144	val_145
-408	val_409
-152	val_153
-348	val_349
-292	val_293
-152	val_153
-256	val_257
-292	val_293
-412	val_413
-156	val_157
-228	val_229
-248	val_249
-244	val_245
-276	val_277
-196	val_197
+428	val_429
+296	val_297
 440	val_441
-308	val_309
+448	val_449
+272	val_273
 468	val_469
-152	val_153
-300	val_301
-244	val_245
-484	val_484
-224	val_224
-128	val_128
-152	val_152
-252	val_252
-292	val_292
-208	val_208
-396	val_396
-128	val_128
-316	val_316
-280	val_280
-208	val_208
-356	val_356
-192	val_192
-176	val_176
-216	val_216
-176	val_176
-332	val_332
-180	val_180
-284	val_284
-260	val_260
-404	val_404
-384	val_384
-272	val_272
+200	val_200
+400	val_400
 348	val_348
-208	val_208
+152	val_152
+448	val_448
+344	val_344
 348	val_348
-172	val_172
-496	val_496
-468	val_468
-120	val_120
-404	val_404
-436	val_436
-156	val_156
-468	val_468
-308	val_308
-196	val_196
-288	val_288
-316	val_316
-364	val_364
-224	val_224
-392	val_392
-272	val_272
-452	val_452
-396	val_396
-336	val_336
-168	val_168
-472	val_472
-160	val_160
 492	val_492
-228	val_228
-468	val_468
-368	val_368
-296	val_296
-216	val_216
-344	val_344
-116	val_116
+384	val_384
 256	val_256
+384	val_384
+172	val_172
+136	val_136
 480	val_480
-288	val_288
-244	val_244
-128	val_128
-432	val_432
-316	val_316
-280	val_280
+460	val_460
+468	val_468
+120	val_120
+444	val_444
+248	val_248
+360	val_360
+200	val_200
 104	val_104
-348	val_348
+480	val_480
 424	val_424
-396	val_396
 164	val_164
 164	val_164
+396	val_396
 424	val_424
-480	val_480
+348	val_348
 104	val_104
-200	val_200
-360	val_360
-248	val_248
-444	val_444
-120	val_120
-468	val_468
-460	val_460
+280	val_280
+316	val_316
+432	val_432
+128	val_128
+244	val_244
+288	val_288
 480	val_480
-136	val_136
-172	val_172
-384	val_384
 256	val_256
-384	val_384
+116	val_116
+344	val_344
+216	val_216
+296	val_296
+368	val_368
+468	val_468
+228	val_228
 492	val_492
+160	val_160
+472	val_472
+168	val_168
+336	val_336
+396	val_396
+452	val_452
+272	val_272
+392	val_392
+224	val_224
+364	val_364
+316	val_316
+288	val_288
+196	val_196
+308	val_308
+468	val_468
+156	val_156
+436	val_436
+404	val_404
+120	val_120
+468	val_468
+496	val_496
+172	val_172
 348	val_348
-344	val_344
-448	val_448
-152	val_152
+208	val_208
 348	val_348
-400	val_400
-200	val_200
+272	val_272
+384	val_384
+404	val_404
+260	val_260
+284	val_284
+180	val_180
+332	val_332
+176	val_176
+216	val_216
+176	val_176
+192	val_192
+356	val_356
+208	val_208
+280	val_280
+316	val_316
+128	val_128
+396	val_396
+208	val_208
+292	val_292
+252	val_252
+152	val_152
+128	val_128
+224	val_224
+484	val_484
+244	val_245
+300	val_301
+152	val_153
+468	val_469
+308	val_309
+440	val_441
+196	val_197
+276	val_277
+244	val_245
+248	val_249
+228	val_229
+156	val_157
+412	val_413
+292	val_293
+256	val_257
+152	val_153
+292	val_293
+348	val_349
+152	val_153
+408	val_409
+144	val_145
+496	val_497
+432	val_433
+136	val_137
+164	val_165
+284	val_285
+244	val_245
+172	val_173
+156	val_157
+436	val_437
+104	val_105
+392	val_393
+268	val_269
+300	val_301
+404	val_405
+384	val_385
+196	val_197
+216	val_217
+204	val_205
+412	val_413
+160	val_161
+352	val_353
+356	val_357
+192	val_193
+132	val_133
+104	val_105
+116	val_117
+384	val_385
+404	val_405
+328	val_329
+260	val_261
+404	val_405
+324	val_325
+384	val_385
+168	val_169
+364	val_365
+416	val_417
+308	val_309
+212	val_213
+140	val_141
+104	val_105
+164	val_165
+260	val_261
+484	val_485
+340	val_341
+336	val_337
+264	val_265
+252	val_253
+476	val_477
+392	val_393
+480	val_481
+480	val_481
+328	val_329
+352	val_353
+284	val_285
+184	val_185
+388	val_389
+132	val_133
+120	val_121
+376	val_377
+492	val_493
+360	val_361
+492	val_493
+196	val_197
+264	val_265
+304	val_305
+344	val_345
+224	val_225
+468	val_469
+128	val_129
+488	val_489
+424	val_425
+476	val_477
+408	val_409
+240	val_241
+128	val_129

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientpositive/sample9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sample9.q.out b/ql/src/test/results/clientpositive/sample9.q.out
index a4c3ff6..f4afad4 100644
--- a/ql/src/test/results/clientpositive/sample9.q.out
+++ b/ql/src/test/results/clientpositive/sample9.q.out
@@ -54,7 +54,7 @@ STAGE PLANS:
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
-            base file name: srcbucket0.txt
+            base file name: 000000_0
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
@@ -99,7 +99,7 @@ STAGE PLANS:
               name: default.srcbucket
             name: default.srcbucket
       Truncated Path -> Alias:
-        /srcbucket/srcbucket0.txt [s:a]
+        /srcbucket/000000_0 [s:a]
 
   Stage: Stage-0
     Fetch Operator
@@ -117,496 +117,496 @@ FROM (SELECT a.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) a) s
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@srcbucket
 #### A masked pattern was here ####
-474	val_475
-62	val_63
-468	val_469
-272	val_273
-448	val_449
-246	val_247
-440	val_441
-278	val_279
-296	val_297
-428	val_429
-126	val_127
-106	val_107
-356	val_357
 490	val_491
-402	val_403
-128	val_129
-10	val_11
-226	val_227
-110	val_111
-0	val_1
-240	val_241
-286	val_287
-408	val_409
-476	val_477
-482	val_483
-48	val_49
-424	val_425
-226	val_227
-494	val_495
-488	val_489
-94	val_95
-50	val_51
-402	val_403
-128	val_129
-468	val_469
-314	val_315
-224	val_225
-344	val_345
-4	val_5
-206	val_207
-114	val_115
-56	val_57
-114	val_115
-254	val_255
-390	val_391
-304	val_305
-264	val_265
-196	val_197
-238	val_239
-20	val_21
-492	val_493
-82	val_83
-58	val_59
-86	val_87
-438	val_439
-360	val_361
-222	val_223
-42	val_43
-338	val_339
-68	val_69
-16	val_17
-492	val_493
-376	val_377
-120	val_121
-306	val_307
-426	val_427
-132	val_133
-446	val_447
-386	val_387
-388	val_389
-184	val_185
-284	val_285
-246	val_247
-262	val_263
-122	val_123
-438	val_439
-390	val_391
-352	val_353
-226	val_227
-328	val_329
-382	val_383
-342	val_343
-480	val_481
-102	val_103
-480	val_481
-318	val_319
-392	val_393
-476	val_477
-258	val_259
-174	val_175
-252	val_253
-114	val_115
-264	val_265
-48	val_49
-336	val_337
-340	val_341
-390	val_391
-484	val_485
-6	val_7
-260	val_261
-2	val_3
-170	val_171
-164	val_165
-118	val_119
-310	val_311
-104	val_105
-80	val_81
-326	val_327
-450	val_451
-140	val_141
-212	val_213
-308	val_309
-30	val_31
-358	val_359
-416	val_417
-42	val_43
-386	val_387
-454	val_455
-364	val_365
-20	val_21
-52	val_53
-40	val_41
-8	val_9
-168	val_169
-384	val_385
-324	val_325
-310	val_311
-206	val_207
-404	val_405
-206	val_207
-226	val_227
-262	val_263
-260	val_261
-328	val_329
-322	val_323
-122	val_123
-404	val_405
-384	val_385
-76	val_77
-116	val_117
-42	val_43
-104	val_105
-406	val_407
-32	val_33
-132	val_133
-192	val_193
-58	val_59
-70	val_71
 356	val_357
-352	val_353
-52	val_53
-330	val_331
-138	val_139
-160	val_161
-454	val_455
-76	val_77
-174	val_175
-412	val_413
-16	val_17
-204	val_205
+106	val_107
 126	val_127
-274	val_275
-374	val_375
-494	val_495
-216	val_217
-470	val_471
-196	val_197
-302	val_303
-450	val_451
-12	val_13
-398	val_399
-334	val_335
-384	val_385
-60	val_61
-442	val_443
-52	val_53
-404	val_405
-446	val_447
-300	val_301
-0	val_1
-268	val_269
-392	val_393
-104	val_105
-436	val_437
-156	val_157
-118	val_119
-172	val_173
-244	val_245
-6	val_7
-284	val_285
-164	val_165
-136	val_137
-462	val_463
-432	val_433
-496	val_497
-144	val_145
-408	val_409
-152	val_153
-382	val_383
-348	val_349
-122	val_123
-292	val_293
-182	val_183
-474	val_475
-310	val_311
-52	val_53
-486	val_487
-152	val_153
-378	val_379
-414	val_415
-256	val_257
-292	val_293
-412	val_413
-40	val_41
-478	val_479
-178	val_179
-100	val_101
-156	val_157
-228	val_229
-22	val_23
-248	val_249
-402	val_403
-62	val_63
-162	val_163
-244	val_245
-276	val_277
-46	val_47
-78	val_79
-134	val_135
-196	val_197
-410	val_411
-82	val_83
+428	val_429
+296	val_297
+278	val_279
 440	val_441
-100	val_101
-308	val_309
-430	val_431
+246	val_247
+448	val_449
+272	val_273
 468	val_469
-152	val_153
-138	val_139
-76	val_77
-300	val_301
-478	val_479
-118	val_119
-178	val_179
-242	val_243
-244	val_245
-238	val_238
-86	val_86
-278	val_278
-98	val_98
-484	val_484
-150	val_150
-224	val_224
-66	val_66
-128	val_128
-146	val_146
-406	val_406
-374	val_374
+62	val_63
+474	val_475
+200	val_200
+400	val_400
+90	val_90
+126	val_126
+222	val_222
+414	val_414
+194	val_194
+348	val_348
 152	val_152
-82	val_82
-166	val_166
-430	val_430
-252	val_252
-292	val_292
-338	val_338
-446	val_446
-394	val_394
-482	val_482
-174	val_174
-494	val_494
-466	val_466
-208	val_208
-174	val_174
-396	val_396
-162	val_162
-266	val_266
-342	val_342
-0	val_0
-128	val_128
-316	val_316
-302	val_302
-438	val_438
-170	val_170
-20	val_20
-378	val_378
-92	val_92
-72	val_72
-4	val_4
-280	val_280
-208	val_208
-356	val_356
-382	val_382
-498	val_498
-386	val_386
-192	val_192
-286	val_286
-176	val_176
-54	val_54
-138	val_138
-216	val_216
-430	val_430
-278	val_278
-176	val_176
-318	val_318
-332	val_332
-180	val_180
-284	val_284
-12	val_12
-230	val_230
-260	val_260
-404	val_404
-384	val_384
-272	val_272
-138	val_138
+448	val_448
+28	val_28
 84	val_84
+344	val_344
+18	val_18
 348	val_348
-466	val_466
-58	val_58
-8	val_8
-230	val_230
-208	val_208
-348	val_348
-24	val_24
-172	val_172
-42	val_42
-158	val_158
-496	val_496
-0	val_0
-322	val_322
-468	val_468
-454	val_454
-100	val_100
-298	val_298
-418	val_418
-96	val_96
-26	val_26
-230	val_230
-120	val_120
-404	val_404
-436	val_436
-156	val_156
-468	val_468
-308	val_308
-196	val_196
-288	val_288
-98	val_98
-282	val_282
-318	val_318
-318	val_318
-470	val_470
-316	val_316
-0	val_0
-490	val_490
-364	val_364
-118	val_118
-134	val_134
-282	val_282
-138	val_138
-238	val_238
-118	val_118
-72	val_72
-90	val_90
-10	val_10
-306	val_306
-224	val_224
-242	val_242
-392	val_392
-272	val_272
-242	val_242
-452	val_452
-226	val_226
-402	val_402
-396	val_396
-58	val_58
-336	val_336
-168	val_168
-34	val_34
-472	val_472
-322	val_322
-498	val_498
-160	val_160
-42	val_42
-430	val_430
+186	val_186
+362	val_362
 458	val_458
-78	val_78
-76	val_76
+146	val_146
+498	val_498
+298	val_298
+100	val_100
 492	val_492
-218	val_218
-228	val_228
-138	val_138
-30	val_30
-64	val_64
-468	val_468
-76	val_76
-74	val_74
-342	val_342
-230	val_230
-368	val_368
-296	val_296
-216	val_216
-344	val_344
-274	val_274
-116	val_116
+462	val_462
+18	val_18
+384	val_384
+134	val_134
+26	val_26
 256	val_256
-70	val_70
+384	val_384
+454	val_454
+406	val_406
+462	val_462
+214	val_214
+172	val_172
+136	val_136
 480	val_480
-288	val_288
-244	val_244
+460	val_460
+310	val_310
+468	val_468
+178	val_178
+478	val_478
+230	val_230
+120	val_120
+444	val_444
+248	val_248
+360	val_360
+200	val_200
+414	val_414
 438	val_438
-128	val_128
-432	val_432
-202	val_202
-316	val_316
-280	val_280
-2	val_2
-80	val_80
-44	val_44
+70	val_70
 104	val_104
-466	val_466
-366	val_366
-406	val_406
-190	val_190
-406	val_406
-114	val_114
-258	val_258
-90	val_90
-262	val_262
-348	val_348
+24	val_24
+480	val_480
+70	val_70
+382	val_382
 424	val_424
-12	val_12
-396	val_396
 164	val_164
-454	val_454
-478	val_478
 298	val_298
+478	val_478
+454	val_454
 164	val_164
+396	val_396
+12	val_12
 424	val_424
-382	val_382
-70	val_70
-480	val_480
-24	val_24
+348	val_348
+262	val_262
+90	val_90
+258	val_258
+114	val_114
+406	val_406
+190	val_190
+406	val_406
+366	val_366
+466	val_466
 104	val_104
-70	val_70
+44	val_44
+80	val_80
+2	val_2
+280	val_280
+316	val_316
+202	val_202
+432	val_432
+128	val_128
 438	val_438
-414	val_414
-200	val_200
-360	val_360
-248	val_248
-444	val_444
-120	val_120
-230	val_230
-478	val_478
-178	val_178
-468	val_468
-310	val_310
-460	val_460
+244	val_244
+288	val_288
 480	val_480
-136	val_136
-172	val_172
-214	val_214
-462	val_462
-406	val_406
-454	val_454
-384	val_384
+70	val_70
 256	val_256
-26	val_26
-134	val_134
-384	val_384
-18	val_18
-462	val_462
+116	val_116
+274	val_274
+344	val_344
+216	val_216
+296	val_296
+368	val_368
+230	val_230
+342	val_342
+74	val_74
+76	val_76
+468	val_468
+64	val_64
+30	val_30
+138	val_138
+228	val_228
+218	val_218
 492	val_492
-100	val_100
-298	val_298
-498	val_498
-146	val_146
+76	val_76
+78	val_78
 458	val_458
-362	val_362
-186	val_186
+430	val_430
+42	val_42
+160	val_160
+498	val_498
+322	val_322
+472	val_472
+34	val_34
+168	val_168
+336	val_336
+58	val_58
+396	val_396
+402	val_402
+226	val_226
+452	val_452
+242	val_242
+272	val_272
+392	val_392
+242	val_242
+224	val_224
+306	val_306
+10	val_10
+90	val_90
+72	val_72
+118	val_118
+238	val_238
+138	val_138
+282	val_282
+134	val_134
+118	val_118
+364	val_364
+490	val_490
+0	val_0
+316	val_316
+470	val_470
+318	val_318
+318	val_318
+282	val_282
+98	val_98
+288	val_288
+196	val_196
+308	val_308
+468	val_468
+156	val_156
+436	val_436
+404	val_404
+120	val_120
+230	val_230
+26	val_26
+96	val_96
+418	val_418
+298	val_298
+100	val_100
+454	val_454
+468	val_468
+322	val_322
+0	val_0
+496	val_496
+158	val_158
+42	val_42
+172	val_172
+24	val_24
+348	val_348
+208	val_208
+230	val_230
+8	val_8
+58	val_58
+466	val_466
 348	val_348
-18	val_18
-344	val_344
 84	val_84
-28	val_28
-448	val_448
+138	val_138
+272	val_272
+384	val_384
+404	val_404
+260	val_260
+230	val_230
+12	val_12
+284	val_284
+180	val_180
+332	val_332
+318	val_318
+176	val_176
+278	val_278
+430	val_430
+216	val_216
+138	val_138
+54	val_54
+176	val_176
+286	val_286
+192	val_192
+386	val_386
+498	val_498
+382	val_382
+356	val_356
+208	val_208
+280	val_280
+4	val_4
+72	val_72
+92	val_92
+378	val_378
+20	val_20
+170	val_170
+438	val_438
+302	val_302
+316	val_316
+128	val_128
+0	val_0
+342	val_342
+266	val_266
+162	val_162
+396	val_396
+174	val_174
+208	val_208
+466	val_466
+494	val_494
+174	val_174
+482	val_482
+394	val_394
+446	val_446
+338	val_338
+292	val_292
+252	val_252
+430	val_430
+166	val_166
+82	val_82
 152	val_152
-348	val_348
-194	val_194
-414	val_414
-222	val_222
-126	val_126
-90	val_90
-400	val_400
-200	val_200
+374	val_374
+406	val_406
+146	val_146
+128	val_128
+66	val_66
+224	val_224
+150	val_150
+484	val_484
+98	val_98
+278	val_278
+86	val_86
+238	val_238
+244	val_245
+242	val_243
+178	val_179
+118	val_119
+478	val_479
+300	val_301
+76	val_77
+138	val_139
+152	val_153
+468	val_469
+430	val_431
+308	val_309
+100	val_101
+440	val_441
+82	val_83
+410	val_411
+196	val_197
+134	val_135
+78	val_79
+46	val_47
+276	val_277
+244	val_245
+162	val_163
+62	val_63
+402	val_403
+248	val_249
+22	val_23
+228	val_229
+156	val_157
+100	val_101
+178	val_179
+478	val_479
+40	val_41
+412	val_413
+292	val_293
+256	val_257
+414	val_415
+378	val_379
+152	val_153
+486	val_487
+52	val_53
+310	val_311
+474	val_475
+182	val_183
+292	val_293
+122	val_123
+348	val_349
+382	val_383
+152	val_153
+408	val_409
+144	val_145
+496	val_497
+432	val_433
+462	val_463
+136	val_137
+164	val_165
+284	val_285
+6	val_7
+244	val_245
+172	val_173
+118	val_119
+156	val_157
+436	val_437
+104	val_105
+392	val_393
+268	val_269
+0	val_1
+300	val_301
+446	val_447
+404	val_405
+52	val_53
+442	val_443
+60	val_61
+384	val_385
+334	val_335
+398	val_399
+12	val_13
+450	val_451
+302	val_303
+196	val_197
+470	val_471
+216	val_217
+494	val_495
+374	val_375
+274	val_275
+126	val_127
+204	val_205
+16	val_17
+412	val_413
+174	val_175
+76	val_77
+454	val_455
+160	val_161
+138	val_139
+330	val_331
+52	val_53
+352	val_353
+356	val_357
+70	val_71
+58	val_59
+192	val_193
+132	val_133
+32	val_33
+406	val_407
+104	val_105
+42	val_43
+116	val_117
+76	val_77
+384	val_385
+404	val_405
+122	val_123
+322	val_323
+328	val_329
+260	val_261
+262	val_263
+226	val_227
+206	val_207
+404	val_405
+206	val_207
+310	val_311
+324	val_325
+384	val_385
+168	val_169
+8	val_9
+40	val_41
+52	val_53
+20	val_21
+364	val_365
+454	val_455
+386	val_387
+42	val_43
+416	val_417
+358	val_359
+30	val_31
+308	val_309
+212	val_213
+140	val_141
+450	val_451
+326	val_327
+80	val_81
+104	val_105
+310	val_311
+118	val_119
+164	val_165
+170	val_171
+2	val_3
+260	val_261
+6	val_7
+484	val_485
+390	val_391
+340	val_341
+336	val_337
+48	val_49
+264	val_265
+114	val_115
+252	val_253
+174	val_175
+258	val_259
+476	val_477
+392	val_393
+318	val_319
+480	val_481
+102	val_103
+480	val_481
+342	val_343
+382	val_383
+328	val_329
+226	val_227
+352	val_353
+390	val_391
+438	val_439
+122	val_123
+262	val_263
+246	val_247
+284	val_285
+184	val_185
+388	val_389
+386	val_387
+446	val_447
+132	val_133
+426	val_427
+306	val_307
+120	val_121
+376	val_377
+492	val_493
+16	val_17
+68	val_69
+338	val_339
+42	val_43
+222	val_223
+360	val_361
+438	val_439
+86	val_87
+58	val_59
+82	val_83
+492	val_493
+20	val_21
+238	val_239
+196	val_197
+264	val_265
+304	val_305
+390	val_391
+254	val_255
+114	val_115
+56	val_57
+114	val_115
+206	val_207
+4	val_5
+344	val_345
+224	val_225
+314	val_315
+468	val_469
+128	val_129
+402	val_403
+50	val_51
+94	val_95
+488	val_489
+494	val_495
+226	val_227
+424	val_425
+48	val_49
+482	val_483
+476	val_477
+408	val_409
+286	val_287
+240	val_241
+0	val_1
+110	val_111
+226	val_227
+10	val_11
+128	val_129
+402	val_403

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientpositive/spark/sample2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/sample2.q.out b/ql/src/test/results/clientpositive/spark/sample2.q.out
index fe9e2f5..fa230aa 100644
--- a/ql/src/test/results/clientpositive/spark/sample2.q.out
+++ b/ql/src/test/results/clientpositive/spark/sample2.q.out
@@ -79,7 +79,7 @@ STAGE PLANS:
             Path -> Partition:
 #### A masked pattern was here ####
                 Partition
-                  base file name: srcbucket0.txt
+                  base file name: 000000_0
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
@@ -124,7 +124,7 @@ STAGE PLANS:
                     name: default.srcbucket
                   name: default.srcbucket
             Truncated Path -> Alias:
-              /srcbucket/srcbucket0.txt [s]
+              /srcbucket/000000_0 [s]
 
   Stage: Stage-0
     Move Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientpositive/spark/sample4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/sample4.q.out b/ql/src/test/results/clientpositive/spark/sample4.q.out
index 987a445..0f2af55 100644
--- a/ql/src/test/results/clientpositive/spark/sample4.q.out
+++ b/ql/src/test/results/clientpositive/spark/sample4.q.out
@@ -79,7 +79,7 @@ STAGE PLANS:
             Path -> Partition:
 #### A masked pattern was here ####
                 Partition
-                  base file name: srcbucket0.txt
+                  base file name: 000000_0
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
@@ -124,7 +124,7 @@ STAGE PLANS:
                     name: default.srcbucket
                   name: default.srcbucket
             Truncated Path -> Alias:
-              /srcbucket/srcbucket0.txt [s]
+              /srcbucket/000000_0 [s]
 
   Stage: Stage-0
     Move Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientpositive/spark/sample6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/sample6.q.out b/ql/src/test/results/clientpositive/spark/sample6.q.out
index 2ed7d7a..1158732 100644
--- a/ql/src/test/results/clientpositive/spark/sample6.q.out
+++ b/ql/src/test/results/clientpositive/spark/sample6.q.out
@@ -77,7 +77,7 @@ STAGE PLANS:
             Path -> Partition:
 #### A masked pattern was here ####
                 Partition
-                  base file name: srcbucket0.txt
+                  base file name: 000000_0
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
@@ -122,7 +122,7 @@ STAGE PLANS:
                     name: default.srcbucket
                   name: default.srcbucket
             Truncated Path -> Alias:
-              /srcbucket/srcbucket0.txt [s]
+              /srcbucket/000000_0 [s]
 
   Stage: Stage-0
     Move Operator
@@ -468,7 +468,7 @@ STAGE PLANS:
             Path -> Partition:
 #### A masked pattern was here ####
                 Partition
-                  base file name: srcbucket1.txt
+                  base file name: 000001_0
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
@@ -513,7 +513,7 @@ STAGE PLANS:
                     name: default.srcbucket
                   name: default.srcbucket
             Truncated Path -> Alias:
-              /srcbucket/srcbucket1.txt [s]
+              /srcbucket/000001_0 [s]
         Reducer 2 
             Needs Tagging: false
             Reduce Operator Tree:
@@ -843,7 +843,7 @@ STAGE PLANS:
             Path -> Partition:
 #### A masked pattern was here ####
                 Partition
-                  base file name: srcbucket0.txt
+                  base file name: 000000_0
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
@@ -888,7 +888,7 @@ STAGE PLANS:
                     name: default.srcbucket
                   name: default.srcbucket
             Truncated Path -> Alias:
-              /srcbucket/srcbucket0.txt [s]
+              /srcbucket/000000_0 [s]
         Reducer 2 
             Needs Tagging: false
             Reduce Operator Tree:
@@ -2400,7 +2400,7 @@ STAGE PLANS:
             Path -> Partition:
 #### A masked pattern was here ####
                 Partition
-                  base file name: srcbucket20.txt
+                  base file name: 000000_0
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
@@ -2446,7 +2446,7 @@ STAGE PLANS:
                   name: default.srcbucket2
 #### A masked pattern was here ####
                 Partition
-                  base file name: srcbucket22.txt
+                  base file name: 000002_0
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
@@ -2491,8 +2491,8 @@ STAGE PLANS:
                     name: default.srcbucket2
                   name: default.srcbucket2
             Truncated Path -> Alias:
-              /srcbucket2/srcbucket20.txt [s]
-              /srcbucket2/srcbucket22.txt [s]
+              /srcbucket2/000000_0 [s]
+              /srcbucket2/000002_0 [s]
         Reducer 2 
             Needs Tagging: false
             Reduce Operator Tree:
@@ -2545,26 +2545,62 @@ POSTHOOK: Input: default@srcbucket2
 2	val_2
 4	val_4
 8	val_8
+10	val_10
+12	val_12
+12	val_12
+18	val_18
+18	val_18
 20	val_20
 24	val_24
 24	val_24
 26	val_26
 26	val_26
 28	val_28
+30	val_30
+34	val_34
 42	val_42
 42	val_42
 44	val_44
+54	val_54
+58	val_58
+58	val_58
 64	val_64
 66	val_66
+70	val_70
+70	val_70
+70	val_70
+72	val_72
+72	val_72
+74	val_74
+76	val_76
+76	val_76
+78	val_78
 80	val_80
 82	val_82
 84	val_84
 84	val_84
 86	val_86
+90	val_90
+90	val_90
+90	val_90
+92	val_92
+96	val_96
+98	val_98
+98	val_98
+100	val_100
+100	val_100
+104	val_104
+104	val_104
 114	val_114
 116	val_116
 118	val_118
 118	val_118
+120	val_120
+120	val_120
+126	val_126
+128	val_128
+128	val_128
+128	val_128
 134	val_134
 134	val_134
 136	val_136
@@ -2572,11 +2608,19 @@ POSTHOOK: Input: default@srcbucket2
 138	val_138
 138	val_138
 138	val_138
+146	val_146
+146	val_146
 150	val_150
 152	val_152
 152	val_152
 156	val_156
 158	val_158
+160	val_160
+162	val_162
+164	val_164
+164	val_164
+166	val_166
+168	val_168
 170	val_170
 172	val_172
 172	val_172
@@ -2585,6 +2629,8 @@ POSTHOOK: Input: default@srcbucket2
 176	val_176
 176	val_176
 178	val_178
+180	val_180
+186	val_186
 190	val_190
 192	val_192
 194	val_194
@@ -2595,18 +2641,38 @@ POSTHOOK: Input: default@srcbucket2
 208	val_208
 208	val_208
 208	val_208
+214	val_214
+216	val_216
+216	val_216
+218	val_218
 222	val_222
 224	val_224
 224	val_224
 226	val_226
 228	val_228
+230	val_230
+230	val_230
+230	val_230
+230	val_230
+230	val_230
+238	val_238
+238	val_238
 242	val_242
 242	val_242
 244	val_244
 248	val_248
+252	val_252
+256	val_256
+256	val_256
+258	val_258
 260	val_260
 262	val_262
 266	val_266
+272	val_272
+272	val_272
+274	val_274
+278	val_278
+278	val_278
 280	val_280
 280	val_280
 282	val_282
@@ -2615,6 +2681,14 @@ POSTHOOK: Input: default@srcbucket2
 286	val_286
 288	val_288
 288	val_288
+292	val_292
+296	val_296
+298	val_298
+298	val_298
+298	val_298
+302	val_302
+306	val_306
+308	val_308
 310	val_310
 316	val_316
 316	val_316
@@ -2622,12 +2696,34 @@ POSTHOOK: Input: default@srcbucket2
 318	val_318
 318	val_318
 318	val_318
+322	val_322
+322	val_322
 332	val_332
 336	val_336
 338	val_338
+342	val_342
+342	val_342
+344	val_344
+344	val_344
+348	val_348
+348	val_348
+348	val_348
+348	val_348
+348	val_348
 356	val_356
+360	val_360
+362	val_362
+364	val_364
+366	val_366
+368	val_368
 374	val_374
 378	val_378
+382	val_382
+382	val_382
+384	val_384
+384	val_384
+384	val_384
+386	val_386
 392	val_392
 394	val_394
 396	val_396
@@ -2641,11 +2737,28 @@ POSTHOOK: Input: default@srcbucket2
 406	val_406
 406	val_406
 406	val_406
+414	val_414
+414	val_414
+418	val_418
 424	val_424
 424	val_424
+430	val_430
+430	val_430
+430	val_430
+432	val_432
+436	val_436
+438	val_438
+438	val_438
+438	val_438
 444	val_444
 446	val_446
 448	val_448
+452	val_452
+454	val_454
+454	val_454
+454	val_454
+458	val_458
+458	val_458
 460	val_460
 462	val_462
 462	val_462
@@ -2656,11 +2769,23 @@ POSTHOOK: Input: default@srcbucket2
 468	val_468
 468	val_468
 468	val_468
+470	val_470
+472	val_472
+478	val_478
+478	val_478
 480	val_480
 480	val_480
 480	val_480
 482	val_482
 484	val_484
+490	val_490
+492	val_492
+492	val_492
+494	val_494
+496	val_496
+498	val_498
+498	val_498
+498	val_498
 PREHOOK: query: EXPLAIN EXTENDED SELECT s.* FROM srcbucket2 TABLESAMPLE (BUCKET 2 OUT OF 4 on key) s
 ORDER BY key, value
 PREHOOK: type: QUERY
@@ -2705,7 +2830,7 @@ STAGE PLANS:
             Path -> Partition:
 #### A masked pattern was here ####
                 Partition
-                  base file name: srcbucket21.txt
+                  base file name: 000001_0
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
@@ -2750,7 +2875,7 @@ STAGE PLANS:
                     name: default.srcbucket2
                   name: default.srcbucket2
             Truncated Path -> Alias:
-              /srcbucket2/srcbucket21.txt [s]
+              /srcbucket2/000001_0 [s]
         Reducer 2 
             Needs Tagging: false
             Reduce Operator Tree:
@@ -2801,22 +2926,103 @@ POSTHOOK: Input: default@srcbucket2
 5	val_5
 5	val_5
 9	val_9
+17	val_17
+33	val_33
+37	val_37
+37	val_37
 41	val_41
+53	val_53
+57	val_57
+65	val_65
+69	val_69
+77	val_77
 85	val_85
+97	val_97
+97	val_97
+105	val_105
+113	val_113
+113	val_113
+125	val_125
+125	val_125
+129	val_129
+129	val_129
 133	val_133
 137	val_137
 137	val_137
+145	val_145
+149	val_149
+149	val_149
+153	val_153
+157	val_157
+165	val_165
+165	val_165
+169	val_169
+169	val_169
+169	val_169
+169	val_169
 177	val_177
+181	val_181
+189	val_189
+193	val_193
+193	val_193
+193	val_193
+197	val_197
+197	val_197
+201	val_201
+205	val_205
+205	val_205
+209	val_209
+209	val_209
+213	val_213
+213	val_213
+217	val_217
+217	val_217
 221	val_221
 221	val_221
 229	val_229
 229	val_229
+233	val_233
+233	val_233
+237	val_237
+237	val_237
+241	val_241
+249	val_249
+257	val_257
 265	val_265
 265	val_265
+273	val_273
+273	val_273
+273	val_273
+277	val_277
+277	val_277
+277	val_277
+277	val_277
+281	val_281
+281	val_281
+285	val_285
+289	val_289
+305	val_305
+309	val_309
+309	val_309
 317	val_317
 317	val_317
+321	val_321
+321	val_321
+325	val_325
+325	val_325
+333	val_333
+333	val_333
+341	val_341
+345	val_345
 353	val_353
 353	val_353
+365	val_365
+369	val_369
+369	val_369
+369	val_369
+373	val_373
+377	val_377
+389	val_389
 393	val_393
 397	val_397
 397	val_397
@@ -2828,13 +3034,32 @@ POSTHOOK: Input: default@srcbucket2
 409	val_409
 409	val_409
 409	val_409
+413	val_413
+413	val_413
+417	val_417
+417	val_417
+417	val_417
+421	val_421
+429	val_429
+429	val_429
+437	val_437
 449	val_449
+453	val_453
+457	val_457
+469	val_469
+469	val_469
+469	val_469
+469	val_469
+469	val_469
+477	val_477
 481	val_481
 485	val_485
 489	val_489
 489	val_489
 489	val_489
 489	val_489
+493	val_493
+497	val_497
 PREHOOK: query: CREATE TABLE empty_bucket (key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default

http://git-wip-us.apache.org/repos/asf/hive/blob/0a6fc638/ql/src/test/results/clientpositive/spark/sample7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/sample7.q.out b/ql/src/test/results/clientpositive/spark/sample7.q.out
index 784000d..22663ac 100644
--- a/ql/src/test/results/clientpositive/spark/sample7.q.out
+++ b/ql/src/test/results/clientpositive/spark/sample7.q.out
@@ -78,7 +78,7 @@ STAGE PLANS:
             Path -> Partition:
 #### A masked pattern was here ####
                 Partition
-                  base file name: srcbucket0.txt
+                  base file name: 000000_0
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
@@ -123,7 +123,7 @@ STAGE PLANS:
                     name: default.srcbucket
                   name: default.srcbucket
             Truncated Path -> Alias:
-              /srcbucket/srcbucket0.txt [s]
+              /srcbucket/000000_0 [s]
 
   Stage: Stage-0
     Move Operator


[24/35] hive git commit: HIVE-14803: S3: Stats gathering for insert queries can be expensive for partitioned dataset (Rajesh Balamohan reviewed by Pengcheng Xiong)

Posted by se...@apache.org.
HIVE-14803: S3: Stats gathering for insert queries can be expensive for partitioned dataset (Rajesh Balamohan reviewed by Pengcheng Xiong)


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

Branch: refs/heads/hive-14535
Commit: 3baca6ceaaff69d2d367cb742251e069a1daaff8
Parents: cf87b0e
Author: Pengcheng Xiong <px...@apache.org>
Authored: Fri Nov 18 14:35:09 2016 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Fri Nov 18 14:35:09 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/StatsTask.java   | 71 +++++++++++++++++++-
 1 file changed, 68 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3baca6ce/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
index 9e528b5..aa5d914 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
@@ -24,6 +24,11 @@ import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -56,6 +61,9 @@ import org.apache.hadoop.hive.ql.stats.StatsFactory;
 import org.apache.hadoop.hive.ql.stats.StatsPublisher;
 import org.apache.hadoop.util.StringUtils;
 
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 /**
  * StatsTask implementation. StatsTask mainly deals with "collectable" stats. These are
  * stats that require data scanning and are collected during query execution (unless the user
@@ -146,7 +154,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
     EnvironmentContext environmentContext = null;
     try {
       // Stats setup:
-      Warehouse wh = new Warehouse(conf);
+      final Warehouse wh = new Warehouse(conf);
       if (!getWork().getNoStatsAggregator() && !getWork().isNoScanAnalyzeCommand()) {
         try {
           scc = getContext();
@@ -216,6 +224,57 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
         // Need to get the old stats of the partition
         // and update the table stats based on the old and new stats.
         List<Partition> updates = new ArrayList<Partition>();
+
+        //Get the file status up-front for all partitions. Beneficial in cases of blob storage systems
+        final Map<String, FileStatus[]> fileStatusMap = new ConcurrentHashMap<String, FileStatus[]>();
+        int poolSize = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 1);
+        // In case thread count is set to 0, use single thread.
+        poolSize = Math.max(poolSize, 1);
+        final ExecutorService pool = Executors.newFixedThreadPool(poolSize,
+          new ThreadFactoryBuilder().setDaemon(true)
+            .setNameFormat("stats-updater-thread-%d")
+            .build());
+        final List<Future<Void>> futures = Lists.newLinkedList();
+        LOG.debug("Getting file stats of all partitions. threadpool size:" + poolSize);
+        try {
+          for(final Partition partn : partitions) {
+            final String partitionName = partn.getName();
+            final org.apache.hadoop.hive.metastore.api.Partition tPart = partn.getTPartition();
+            Map<String, String> parameters = tPart.getParameters();
+
+            if (!existStats(parameters) && atomic) {
+              continue;
+            }
+            futures.add(pool.submit(new Callable<Void>() {
+              @Override
+              public Void call() throws Exception {
+                FileStatus[] partfileStatus = wh.getFileStatusesForSD(tPart.getSd());
+                fileStatusMap.put(partitionName,  partfileStatus);
+                return null;
+              }
+            }));
+          }
+          pool.shutdown();
+          for(Future<Void> future : futures) {
+            future.get();
+          }
+        } catch (InterruptedException e) {
+          LOG.debug("Cancelling " + futures.size() + " file stats lookup tasks");
+          //cancel other futures
+          for (Future future : futures) {
+            future.cancel(true);
+          }
+          // Fail the query if the stats are supposed to be reliable
+          if (work.isStatsReliable()) {
+            ret = 1;
+          }
+        } finally {
+          if (pool != null) {
+            pool.shutdownNow();
+          }
+          LOG.debug("Finished getting file stats of all partitions");
+        }
+
         for (Partition partn : partitions) {
           //
           // get the old partition stats
@@ -230,7 +289,8 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
                   .getDestinationCreateTable().isEmpty())) {
             StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.TRUE);
           }
-          if (!existStats(parameters) && atomic) {
+          //only when the stats exist, it is added to fileStatusMap
+          if (!fileStatusMap.containsKey(partn.getName())) {
             continue;
           }
 
@@ -241,7 +301,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
             StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.FALSE);
           }
 
-          updateQuickStats(wh, parameters, tPart.getSd());
+          updateQuickStats(parameters, fileStatusMap.get(partn.getName()));
           if (StatsSetupConst.areBasicStatsUptoDate(parameters)) {
             if (statsAggregator != null) {
               String prefix = getAggregationPrefix(table, partn);
@@ -371,6 +431,11 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
      * calculate fast statistics
      */
     FileStatus[] partfileStatus = wh.getFileStatusesForSD(desc);
+    updateQuickStats(parameters, partfileStatus);
+  }
+
+  private void updateQuickStats(Map<String, String> parameters,
+      FileStatus[] partfileStatus) throws MetaException {
     MetaStoreUtils.populateQuickStats(partfileStatus, parameters);
   }
 


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

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

Conflicts:
	ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
	ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java


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

Branch: refs/heads/hive-14535
Commit: 536e9b004d79f506c44e524d6f89f9c10433eafb
Parents: 13ea883 893b255
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Nov 21 18:38:43 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Nov 21 18:38:43 2016 -0800

----------------------------------------------------------------------
 .../org/apache/hive/beeline/HiveSchemaTool.java |  187 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   19 +-
 data/files/parquet_non_dictionary_types.txt     |  300 +++
 data/scripts/q_test_init.sql                    |   20 +-
 .../hive/hbase/HiveHFileOutputFormat.java       |   13 +-
 .../src/test/queries/positive/hbase_bulk.q      |    2 +-
 .../test/queries/positive/hbase_handler_bulk.q  |    3 +
 .../hive/hcatalog/common/HCatConstants.java     |    5 +
 .../listener/DbNotificationListener.java        |   77 +-
 .../hcatalog/messaging/AlterIndexMessage.java   |   30 +
 .../messaging/CreateFunctionMessage.java        |   30 +
 .../hcatalog/messaging/CreateIndexMessage.java  |   30 +
 .../hcatalog/messaging/DropFunctionMessage.java |   30 +
 .../hcatalog/messaging/DropIndexMessage.java    |   30 +
 .../hcatalog/messaging/HCatEventMessage.java    |   10 +-
 .../hcatalog/messaging/MessageDeserializer.java |   39 +-
 .../hive/hcatalog/messaging/MessageFactory.java |   38 +
 .../messaging/json/JSONAlterIndexMessage.java   |   89 +
 .../json/JSONCreateFunctionMessage.java         |   81 +
 .../messaging/json/JSONCreateIndexMessage.java  |   82 +
 .../messaging/json/JSONDropFunctionMessage.java |   81 +
 .../messaging/json/JSONDropIndexMessage.java    |   82 +
 .../messaging/json/JSONMessageDeserializer.java |   56 +-
 .../messaging/json/JSONMessageFactory.java      |   62 +-
 .../e2e/templeton/drivers/TestDriverCurl.pm     |    2 +-
 .../listener/DummyRawStoreFailEvent.java        |   30 +-
 .../listener/TestDbNotificationListener.java    |  286 +-
 .../test/queries/clientpositive/insert_into.q   |    3 +
 .../results/clientpositive/insert_into.q.out    |  299 ++-
 .../org/apache/hive/beeline/TestSchemaTool.java |   74 +-
 .../test/resources/testconfiguration.properties |    1 +
 .../control/AbstractCoreBlobstoreCliDriver.java |    2 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   30 +-
 .../hive/llap/cli/LlapOptionsProcessor.java     |    6 +
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |  685 ++---
 .../hive/llap/io/api/impl/LlapInputFormat.java  |    5 -
 .../llap/io/decode/OrcEncodedDataConsumer.java  |  197 +-
 .../llap/io/encoded/OrcEncodedDataReader.java   |   89 +-
 .../hive/llap/io/metadata/OrcFileMetadata.java  |    6 +
 .../llap/io/metadata/OrcStripeMetadata.java     |   11 +-
 llap-server/src/main/resources/package.py       |   16 +
 llap-server/src/main/resources/templates.py     |    3 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   51 +-
 .../hive/metastore/MetaStoreEventListener.java  |   16 +
 .../metastore/events/CreateFunctionEvent.java   |   39 +
 .../metastore/events/DropFunctionEvent.java     |   39 +
 .../hadoop/hive/metastore/DummyListener.java    |   12 +
 .../orc/impl/ConvertTreeReaderFactory.java      |    2 +-
 .../org/apache/orc/impl/TreeReaderFactory.java  |   90 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |    5 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |    8 +-
 .../exec/ExprNodeConstantDefaultEvaluator.java  |   50 +
 .../hive/ql/exec/ExprNodeEvaluatorFactory.java  |    7 +
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |    5 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |    2 +
 .../apache/hadoop/hive/ql/exec/StatsTask.java   |   71 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |    6 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |   42 +-
 .../hadoop/hive/ql/io/merge/MergeFileTask.java  |    4 +
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |    2 +-
 .../hive/ql/io/orc/encoded/EncodedReader.java   |    2 +-
 .../ql/io/orc/encoded/EncodedReaderImpl.java    |   80 +-
 .../orc/encoded/EncodedTreeReaderFactory.java   |  985 +++++--
 .../hadoop/hive/ql/io/orc/encoded/Reader.java   |    6 +-
 .../ql/io/parquet/MapredParquetInputFormat.java |   10 +-
 .../ql/io/parquet/ParquetRecordReaderBase.java  |  171 ++
 .../parquet/VectorizedParquetInputFormat.java   |  136 +-
 .../parquet/read/DataWritableReadSupport.java   |    8 +-
 .../read/ParquetRecordReaderWrapper.java        |  141 +-
 .../ql/io/parquet/timestamp/NanoTimeUtils.java  |    2 +-
 .../parquet/vector/VectorizedColumnReader.java  |  571 ++++
 .../vector/VectorizedParquetRecordReader.java   |  289 +++
 .../rules/HiveAggregateProjectMergeRule.java    |   14 +
 .../rules/HiveExpandDistinctAggregatesRule.java |  206 +-
 .../JoinCondTypeCheckProcFactory.java           |  116 +-
 .../ql/optimizer/ppr/PartExprEvalUtils.java     |    8 +-
 .../hive/ql/optimizer/ppr/PartitionPruner.java  |   19 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    5 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   23 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   14 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |   19 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |   16 +-
 .../apache/hadoop/hive/ql/parse/QBJoinTree.java |    2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   94 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java      |   11 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |   82 -
 .../ql/plan/ExprNodeConstantDefaultDesc.java    |   86 +
 .../apache/hadoop/hive/ql/plan/FetchWork.java   |   14 +
 .../hadoop/hive/ql/plan/FileSinkDesc.java       |   14 +
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |   41 +-
 .../org/apache/hadoop/hive/ql/udf/UDFUUID.java  |    1 +
 .../udf/generic/GenericUDFInternalInterval.java |  322 +++
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |    9 +
 .../ql/udf/generic/GenericUDFOPNotEqual.java    |    9 +
 .../ql/exec/tez/TestCustomPartitionVertex.java  |   17 +
 .../io/parquet/TestVectorizedColumnReader.java  |  429 +++
 .../hive/ql/parse/TestParseDriverIntervals.java |   83 +
 .../apache/hadoop/hive/ql/udf/TestUDFUUID.java  |    5 +
 .../generic/TestGenericUDFInternalInterval.java |  144 +
 .../alter_concatenate_indexed_table.q           |    2 +
 .../clientnegative/alter_view_failure6.q        |    2 +
 .../clientnegative/bucket_mapjoin_mismatch1.q   |    2 +
 .../bucket_mapjoin_wrong_table_metadata_1.q     |    2 +
 .../bucket_mapjoin_wrong_table_metadata_2.q     |    2 +
 .../clientnegative/compare_double_bigint.q      |    2 +
 .../clientnegative/compare_string_bigint.q      |    2 +
 .../dynamic_partitions_with_whitelist.q         |    2 +
 .../exim_11_nonpart_noncompat_sorting.q         |    2 +
 ql/src/test/queries/clientnegative/input4.q     |    2 +
 .../queries/clientnegative/input_part0_neg.q    |    2 +
 .../insertover_dynapart_ifnotexists.q           |    2 +
 ql/src/test/queries/clientnegative/join45.q     |   13 +
 .../test/queries/clientnegative/strict_join.q   |    2 +
 .../queries/clientnegative/strict_orderby.q     |    2 +
 .../queries/clientnegative/strict_pruning.q     |    2 +
 .../alter_concatenate_indexed_table.q           |    2 +
 .../test/queries/clientpositive/alter_merge.q   |    2 +
 .../test/queries/clientpositive/alter_merge_2.q |    2 +
 .../queries/clientpositive/alter_merge_stats.q  |    2 +
 .../queries/clientpositive/autoColumnStats_1.q  |    2 +
 .../queries/clientpositive/autoColumnStats_2.q  |    2 +
 .../clientpositive/auto_sortmerge_join_1.q      |    2 +
 .../clientpositive/auto_sortmerge_join_11.q     |    2 +
 .../clientpositive/auto_sortmerge_join_12.q     |    2 +
 .../clientpositive/auto_sortmerge_join_16.q     |    2 +
 .../clientpositive/auto_sortmerge_join_2.q      |    2 +
 .../clientpositive/auto_sortmerge_join_3.q      |    2 +
 .../clientpositive/auto_sortmerge_join_4.q      |    2 +
 .../clientpositive/auto_sortmerge_join_5.q      |    2 +
 .../clientpositive/auto_sortmerge_join_7.q      |    2 +
 .../clientpositive/auto_sortmerge_join_8.q      |    2 +
 .../queries/clientpositive/bucket_map_join_1.q  |    2 +
 .../queries/clientpositive/bucket_map_join_2.q  |    2 +
 .../clientpositive/bucket_map_join_spark1.q     |    2 +
 .../clientpositive/bucket_map_join_spark2.q     |    2 +
 .../clientpositive/bucket_map_join_spark3.q     |    2 +
 .../clientpositive/bucket_map_join_tez1.q       |    2 +
 .../clientpositive/bucket_map_join_tez2.q       |    2 +
 .../queries/clientpositive/bucketcontext_1.q    |    2 +
 .../queries/clientpositive/bucketcontext_2.q    |    2 +
 .../queries/clientpositive/bucketcontext_3.q    |    2 +
 .../queries/clientpositive/bucketcontext_4.q    |    2 +
 .../queries/clientpositive/bucketcontext_5.q    |    2 +
 .../queries/clientpositive/bucketcontext_6.q    |    2 +
 .../queries/clientpositive/bucketcontext_7.q    |    2 +
 .../queries/clientpositive/bucketcontext_8.q    |    2 +
 .../bucketizedhiveinputformat_auto.q            |    2 +
 .../queries/clientpositive/bucketmapjoin1.q     |    2 +
 .../queries/clientpositive/bucketmapjoin10.q    |    2 +
 .../queries/clientpositive/bucketmapjoin11.q    |    2 +
 .../queries/clientpositive/bucketmapjoin12.q    |    2 +
 .../queries/clientpositive/bucketmapjoin2.q     |    2 +
 .../queries/clientpositive/bucketmapjoin3.q     |    2 +
 .../queries/clientpositive/bucketmapjoin4.q     |    2 +
 .../queries/clientpositive/bucketmapjoin5.q     |    2 +
 .../queries/clientpositive/bucketmapjoin7.q     |    2 +
 .../queries/clientpositive/bucketmapjoin8.q     |    2 +
 .../queries/clientpositive/bucketmapjoin9.q     |    2 +
 .../clientpositive/bucketmapjoin_negative.q     |    2 +
 .../clientpositive/bucketmapjoin_negative2.q    |    2 +
 .../clientpositive/bucketmapjoin_negative3.q    |    2 +
 .../cbo_rp_udaf_percentile_approx_23.q          |    2 +
 ql/src/test/queries/clientpositive/char_1.q     |    2 +
 ql/src/test/queries/clientpositive/char_serde.q |    2 +
 .../clientpositive/create_merge_compressed.q    |    2 +
 ql/src/test/queries/clientpositive/cteViews.q   |    8 +
 .../clientpositive/drop_partitions_filter4.q    |   29 +
 .../queries/clientpositive/explainanalyze_2.q   |    2 +
 .../queries/clientpositive/explainanalyze_3.q   |    2 +
 .../test/queries/clientpositive/explainuser_1.q |    2 +
 .../test/queries/clientpositive/explainuser_2.q |    2 +
 .../test/queries/clientpositive/explainuser_3.q |    2 +
 .../test/queries/clientpositive/global_limit.q  |    2 +
 .../clientpositive/infer_bucket_sort_dyn_part.q |    2 +
 .../queries/clientpositive/insert_orig_table.q  |    2 +
 .../clientpositive/insert_values_orig_table.q   |    2 +
 .../insert_values_orig_table_use_metadata.q     |    2 +
 .../test/queries/clientpositive/interval_alt.q  |   36 +
 ql/src/test/queries/clientpositive/join45.q     |  203 ++
 .../clientpositive/merge_dynamic_partition.q    |    2 +
 .../clientpositive/merge_dynamic_partition2.q   |    2 +
 .../clientpositive/merge_dynamic_partition3.q   |    2 +
 .../clientpositive/merge_dynamic_partition4.q   |    2 +
 .../clientpositive/merge_dynamic_partition5.q   |    2 +
 ql/src/test/queries/clientpositive/mergejoin.q  |    2 +
 .../clientpositive/multi_count_distinct.q       |   38 +
 .../clientpositive/nonReservedKeyWords.q        |    8 +
 ...ypes_non_dictionary_encoding_vectorization.q |   94 +
 .../parquet_types_vectorization.q               |   96 +
 .../test/queries/clientpositive/smb_mapjoin_1.q |    2 +
 .../queries/clientpositive/smb_mapjoin_10.q     |    2 +
 .../test/queries/clientpositive/smb_mapjoin_2.q |    2 +
 .../queries/clientpositive/smb_mapjoin_25.q     |    2 +
 .../test/queries/clientpositive/smb_mapjoin_3.q |    2 +
 .../test/queries/clientpositive/smb_mapjoin_4.q |    2 +
 .../test/queries/clientpositive/smb_mapjoin_5.q |    2 +
 .../test/queries/clientpositive/smb_mapjoin_7.q |    2 +
 .../test/queries/clientpositive/smb_mapjoin_8.q |    2 +
 ql/src/test/queries/clientpositive/stats1.q     |    2 +
 ql/src/test/queries/clientpositive/stats11.q    |    2 +
 ql/src/test/queries/clientpositive/stats18.q    |    2 +
 ql/src/test/queries/clientpositive/tez_fsstat.q |    2 +
 ql/src/test/queries/clientpositive/tez_smb_1.q  |    2 +
 .../test/queries/clientpositive/tez_smb_empty.q |    2 +
 .../test/queries/clientpositive/tez_smb_main.q  |    2 +
 .../clientpositive/udaf_percentile_approx_20.q  |    2 +
 .../clientpositive/udaf_percentile_approx_23.q  |    2 +
 ql/src/test/queries/clientpositive/varchar_1.q  |    2 +
 .../test/queries/clientpositive/varchar_serde.q |    2 +
 .../queries/clientpositive/vector_complex_all.q |   52 +-
 .../clientnegative/alter_view_failure6.q.out    |    2 +-
 .../clientnegative/compare_double_bigint.q.out  |    2 +-
 .../clientnegative/compare_string_bigint.q.out  |    2 +-
 ql/src/test/results/clientnegative/input4.q.out |    2 +-
 .../clientnegative/input_part0_neg.q.out        |    2 +-
 ql/src/test/results/clientnegative/join45.q.out |   13 +
 .../results/clientnegative/strict_join.q.out    |    2 +-
 .../results/clientnegative/strict_orderby.q.out |    2 +-
 .../results/clientnegative/strict_pruning.q.out |    2 +-
 .../test/results/clientpositive/cteViews.q.out  |   46 +
 .../drop_partitions_filter4.q.out               |  167 ++
 .../results/clientpositive/interval_alt.q.out   |  135 +
 ql/src/test/results/clientpositive/join45.q.out | 1771 +++++++++++++
 .../llap/vector_complex_all.q.out               |  315 +--
 .../llap/vector_complex_join.q.out              |    4 +-
 .../llap/vector_grouping_sets.q.out             |    8 +-
 .../clientpositive/llap/vector_interval_1.q.out |    6 +-
 .../llap/vector_interval_arithmetic.q.out       |   17 +-
 .../llap/vectorized_parquet.q.out               |    2 +-
 .../llap/vectorized_parquet_types.q.out         |   10 +-
 .../clientpositive/nonReservedKeyWords.q.out    |   51 +
 ..._non_dictionary_encoding_vectorization.q.out | 2452 ++++++++++++++++++
 .../parquet_types_vectorization.q.out           |  850 ++++++
 .../results/clientpositive/perf/query70.q.out   |   52 +-
 .../test/results/clientpositive/sample2.q.out   |  934 +++----
 .../test/results/clientpositive/sample4.q.out   |  934 +++----
 .../test/results/clientpositive/sample6.q.out   |  701 +++--
 .../test/results/clientpositive/sample7.q.out   |  362 +--
 .../test/results/clientpositive/sample9.q.out   |  934 +++----
 .../results/clientpositive/show_functions.q.out |    1 +
 .../results/clientpositive/spark/sample2.q.out  |    4 +-
 .../results/clientpositive/spark/sample4.q.out  |    4 +-
 .../results/clientpositive/spark/sample6.q.out  |  249 +-
 .../results/clientpositive/spark/sample7.q.out  |    4 +-
 .../tez/multi_count_distinct.q.out              |  193 ++
 .../clientpositive/vector_complex_all.q.out     |  302 ++-
 .../clientpositive/vector_grouping_sets.q.out   |    8 +-
 .../clientpositive/vector_interval_1.q.out      |    6 +-
 .../vector_interval_arithmetic.q.out            |   17 +-
 .../vectorized_parquet_types.q.out              |    3 +
 .../org/apache/hive/tmpl/QueryProfileTmpl.jamon |    4 +-
 .../service/cli/operation/SQLOperation.java     |    5 +-
 .../hive-webapps/hiveserver2/hiveserver2.jsp    |    5 +-
 .../common/io/encoded/EncodedColumnBatch.java   |   52 +-
 254 files changed, 15579 insertions(+), 3633 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/hive/blob/536e9b00/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/536e9b00/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/536e9b00/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/536e9b00/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/536e9b00/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
index 3df5d5a,bda94ff..04e8cac
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
@@@ -216,13 -219,12 +220,13 @@@ public class LoadSemanticAnalyzer exten
          && (ts.partSpec == null || ts.partSpec.size() == 0)) {
        throw new SemanticException(ErrorMsg.NEED_PARTITION_ERROR.getMsg());
      }
-     /* TODO# enable later - fails srcbucket creation in 14990
++
      List<String> bucketCols = ts.tableHandle.getBucketCols();
-     if (bucketCols != null && !bucketCols.isEmpty()
-         && MetaStoreUtils.isInsertOnlyTable(ts.tableHandle.getMetadata())) {
-       throw new SemanticException("Cannot load into a bucketed insert-only table. Please load into"
-           + " an intermediate table and use insert... select to allow Hive to enforce bucketing.");
-     }*/
+     if (bucketCols != null && !bucketCols.isEmpty()) {
+       String error = StrictChecks.checkBucketing(conf);
+       if (error != null) throw new SemanticException("Please load into an intermediate table"
+           + " and use 'insert... select' to allow Hive to enforce bucketing. " + error);
+     }
  
      // make sure the arguments make sense
      List<FileStatus> files = applyConstraintsAndGetFiles(fromURI, fromTree, isLocal);

http://git-wip-us.apache.org/repos/asf/hive/blob/536e9b00/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 46f657f,3bc6fe4..31bee14
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@@ -6850,129 -6972,24 +6845,139 @@@ public class SemanticAnalyzer extends B
        genPartnCols(dest, input, qb, table_desc, dest_tab, rsCtx);
      }
  
 -    FileSinkDesc fileSinkDesc = new FileSinkDesc(
 -      queryTmpdir,
 -      table_desc,
 -      conf.getBoolVar(HiveConf.ConfVars.COMPRESSRESULT),
 -      currentTableId,
 -      rsCtx.isMultiFileSpray(),
 -      canBeMerged,
 -      rsCtx.getNumFiles(),
 -      rsCtx.getTotalFiles(),
 -      rsCtx.getPartnCols(),
 -      dpCtx,
 -      dest_path);
 +    assert isMmTable == (mmWriteId != null);
 +    FileSinkDesc fileSinkDesc = createFileSinkDesc(dest, table_desc, dest_part,
 +        dest_path, currentTableId, destTableIsAcid, destTableIsTemporary,
 +        destTableIsMaterialization, queryTmpdir, rsCtx, dpCtx, lbCtx, fsRS,
 +        canBeMerged, mmWriteId, isMmCtas);
 +    if (isMmCtas) {
 +      // Add FSD so that the LoadTask compilation could fix up its path to avoid the move.
 +      tableDesc.setWriter(fileSinkDesc);
 +    }
 +
++    if (SessionState.get().isHiveServerQuery() &&
++      null != table_desc &&
++      table_desc.getSerdeClassName().equalsIgnoreCase(ThriftJDBCBinarySerDe.class.getName()) &&
++      HiveConf.getBoolVar(conf,HiveConf.ConfVars.HIVE_SERVER2_THRIFT_RESULTSET_SERIALIZE_IN_TASKS)) {
++        fileSinkDesc.setIsUsingThriftJDBCBinarySerDe(true);
++    } else {
++        fileSinkDesc.setIsUsingThriftJDBCBinarySerDe(false);
++    }
++
 +    Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
 +        fileSinkDesc, fsRS, input), inputRR);
 +
 +    handleLineage(ltd, output);
 +
 +    if (LOG.isDebugEnabled()) {
 +      LOG.debug("Created FileSink Plan for clause: " + dest + "dest_path: "
 +          + dest_path + " row schema: " + inputRR.toString());
 +    }
 +
 +    FileSinkOperator fso = (FileSinkOperator) output;
 +    fso.getConf().setTable(dest_tab);
 +    // the following code is used to collect column stats when
 +    // hive.stats.autogather=true
 +    // and it is an insert overwrite or insert into table
 +    if (dest_tab != null && conf.getBoolVar(ConfVars.HIVESTATSAUTOGATHER)
 +        && conf.getBoolVar(ConfVars.HIVESTATSCOLAUTOGATHER)
 +        && ColumnStatsAutoGatherContext.canRunAutogatherStats(fso)) {
 +      if (dest_type.intValue() == QBMetaData.DEST_TABLE) {
 +        genAutoColumnStatsGatheringPipeline(qb, table_desc, partSpec, input, qb.getParseInfo()
 +            .isInsertIntoTable(dest_tab.getDbName(), dest_tab.getTableName()));
 +      } else if (dest_type.intValue() == QBMetaData.DEST_PARTITION) {
 +        genAutoColumnStatsGatheringPipeline(qb, table_desc, dest_part.getSpec(), input, qb
 +            .getParseInfo().isInsertIntoTable(dest_tab.getDbName(), dest_tab.getTableName()));
 +
 +      }
 +    }
 +    return output;
 +  }
 +
 +  private ColsAndTypes deriveFileSinkColTypes(
 +      RowResolver inputRR, List<FieldSchema> field_schemas) throws SemanticException {
 +    ColsAndTypes result = new ColsAndTypes("", "");
 +    ArrayList<ColumnInfo> colInfos = inputRR.getColumnInfos();
 +    boolean first = true;
 +    for (ColumnInfo colInfo : colInfos) {
 +      String[] nm = inputRR.reverseLookup(colInfo.getInternalName());
 +
 +      if (nm[1] != null) { // non-null column alias
 +        colInfo.setAlias(nm[1]);
 +      }
 +
 +      String colName = colInfo.getInternalName();  //default column name
 +      if (field_schemas != null) {
 +        FieldSchema col = new FieldSchema();
 +        if (!("".equals(nm[0])) && nm[1] != null) {
 +          colName = unescapeIdentifier(colInfo.getAlias()).toLowerCase(); // remove ``
 +        }
 +        colName = fixCtasColumnName(colName);
 +        col.setName(colName);
 +        String typeName = colInfo.getType().getTypeName();
 +        // CTAS should NOT create a VOID type
 +        if (typeName.equals(serdeConstants.VOID_TYPE_NAME)) {
 +            throw new SemanticException(ErrorMsg.CTAS_CREATES_VOID_TYPE.getMsg(colName));
 +        }
 +        col.setType(typeName);
 +        field_schemas.add(col);
 +      }
 +
 +      if (!first) {
 +        result.cols = result.cols.concat(",");
 +        result.colTypes = result.colTypes.concat(":");
 +      }
 +
 +      first = false;
 +      result.cols = result.cols.concat(colName);
 +
 +      // Replace VOID type with string when the output is a temp table or
 +      // local files.
 +      // A VOID type can be generated under the query:
 +      //
 +      // select NULL from tt;
 +      // or
 +      // insert overwrite local directory "abc" select NULL from tt;
 +      //
 +      // where there is no column type to which the NULL value should be
 +      // converted.
 +      //
 +      String tName = colInfo.getType().getTypeName();
 +      if (tName.equals(serdeConstants.VOID_TYPE_NAME)) {
 +        result.colTypes = result.colTypes.concat(serdeConstants.STRING_TYPE_NAME);
 +      } else {
 +        result.colTypes = result.colTypes.concat(tName);
 +      }
 +    }
 +    return result;
 +  }
 +
 +  private static Long getMmWriteId(Table tbl, boolean isMmTable) throws HiveException {
 +    if (!isMmTable) return null;
 +    // Get the next write ID for this table. We will prefix files with this write ID.
 +    return Hive.get().getNextTableWriteId(tbl.getDbName(), tbl.getTableName());
 +  }
 +
 +  private FileSinkDesc createFileSinkDesc(String dest, TableDesc table_desc,
 +      Partition dest_part, Path dest_path, int currentTableId,
 +      boolean destTableIsAcid, boolean destTableIsTemporary,
 +      boolean destTableIsMaterialization, Path queryTmpdir,
 +      SortBucketRSCtx rsCtx, DynamicPartitionCtx dpCtx, ListBucketingCtx lbCtx,
 +      RowSchema fsRS, boolean canBeMerged, Long mmWriteId, boolean isMmCtas) throws SemanticException {
 +    FileSinkDesc fileSinkDesc = new FileSinkDesc(queryTmpdir, table_desc,
 +        conf.getBoolVar(HiveConf.ConfVars.COMPRESSRESULT), currentTableId, rsCtx.isMultiFileSpray(),
 +        canBeMerged, rsCtx.getNumFiles(), rsCtx.getTotalFiles(), rsCtx.getPartnCols(), dpCtx,
 +        dest_path, mmWriteId, isMmCtas);
  
-     fileSinkDesc.setHiveServerQuery(SessionState.get().isHiveServerQuery());
+     boolean isHiveServerQuery = SessionState.get().isHiveServerQuery();
+     fileSinkDesc.setHiveServerQuery(isHiveServerQuery);
      // If this is an insert, update, or delete on an ACID table then mark that so the
      // FileSinkOperator knows how to properly write to it.
 -    if (destTableIsAcid) {
 +    boolean isDestInsertOnly = (dest_part != null && dest_part.getTable() != null &&
 +        MetaStoreUtils.isInsertOnlyTable(dest_part.getTable().getParameters()))
 +        || (table_desc != null && MetaStoreUtils.isInsertOnlyTable(table_desc.getProperties()));
 +
 +    if (destTableIsAcid && !isDestInsertOnly) {
        AcidUtils.Operation wt = updating(dest) ? AcidUtils.Operation.UPDATE :
            (deleting(dest) ? AcidUtils.Operation.DELETE : AcidUtils.Operation.INSERT);
        fileSinkDesc.setWriteType(wt);

http://git-wip-us.apache.org/repos/asf/hive/blob/536e9b00/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/536e9b00/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
index 504a6ca,4d9139b..d42715a
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
@@@ -96,10 -96,13 +96,16 @@@ public class FileSinkDesc extends Abstr
    private transient Table table;
    private Path destPath;
    private boolean isHiveServerQuery;
 +  private Long mmWriteId;
 +  private boolean isMerge;
 +  private boolean isMmCtas;
  
+   /**
+    * Whether is a HiveServer query, and the destination table is
+    * indeed written using ThriftJDBCBinarySerDe
+    */
+   private boolean isUsingThriftJDBCBinarySerDe = false;
+ 
    public FileSinkDesc() {
    }
  

http://git-wip-us.apache.org/repos/asf/hive/blob/536e9b00/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
----------------------------------------------------------------------


[12/35] hive git commit: HIVE-14815: Implement Parquet vectorization reader for Primitive types(Ferdinand Xu, review by Chao Sun) This closes #104

Posted by se...@apache.org.
HIVE-14815: Implement Parquet vectorization reader for Primitive types(Ferdinand Xu, review by Chao Sun) This closes #104


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

Branch: refs/heads/hive-14535
Commit: 936df7a15a3ce323300cabe7b2ebb90e22f2069d
Parents: 67c022f
Author: Ferdinand Xu <ch...@intel.com>
Authored: Fri Nov 18 08:17:39 2016 +0800
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Fri Nov 18 08:17:39 2016 +0800

----------------------------------------------------------------------
 data/files/parquet_non_dictionary_types.txt     |  300 +++
 .../ql/io/parquet/MapredParquetInputFormat.java |   10 +-
 .../ql/io/parquet/ParquetRecordReaderBase.java  |  171 ++
 .../parquet/VectorizedParquetInputFormat.java   |  136 +-
 .../parquet/read/DataWritableReadSupport.java   |    8 +-
 .../read/ParquetRecordReaderWrapper.java        |  141 +-
 .../ql/io/parquet/timestamp/NanoTimeUtils.java  |    2 +-
 .../parquet/vector/VectorizedColumnReader.java  |  571 ++++
 .../vector/VectorizedParquetRecordReader.java   |  289 +++
 .../io/parquet/TestVectorizedColumnReader.java  |  429 +++
 ...ypes_non_dictionary_encoding_vectorization.q |   94 +
 .../parquet_types_vectorization.q               |   96 +
 .../llap/vectorized_parquet.q.out               |    2 +-
 .../llap/vectorized_parquet_types.q.out         |   10 +-
 ..._non_dictionary_encoding_vectorization.q.out | 2452 ++++++++++++++++++
 .../parquet_types_vectorization.q.out           |  850 ++++++
 .../vectorized_parquet_types.q.out              |    3 +
 17 files changed, 5284 insertions(+), 280 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/data/files/parquet_non_dictionary_types.txt
----------------------------------------------------------------------
diff --git a/data/files/parquet_non_dictionary_types.txt b/data/files/parquet_non_dictionary_types.txt
new file mode 100644
index 0000000..50ea5d2
--- /dev/null
+++ b/data/files/parquet_non_dictionary_types.txt
@@ -0,0 +1,300 @@
+1000|-128|0|0.0|0.3||1940-01-01 01:01:01.111111111||||:|1000,1001|1000,b|1940-01-01|0.0
+1001|-127|1|0.3|1.3|b|1941-02-02 01:01:01.111111111|b|b|b|b:b|1001,1002|1001,c|1941-02-02|0.3
+1002|-126|2|0.6|2.3|c|1942-03-03 01:01:01.111111111|c|c|c|c:c|1002,1003|1002,d|1942-03-03|0.6
+1003|-125|3|0.9|3.3|d|1943-04-04 01:01:01.111111111|d|d|d|d:d|1003,1004|1003,e|1943-04-04|0.9
+1004|-124|4|1.2|4.3|e|1944-05-05 01:01:01.111111111|e|e|e|e:e|1004,1005|1004,f|1944-05-05|1.2
+1005|-123|5|1.5|5.3|f|1945-06-06 01:01:01.111111111|f|f|f|f:f|1005,1006|1005,g|1945-06-06|1.5
+1006|-122|6|1.8|6.3|g|1946-07-07 01:01:01.111111111|g|g|g|g:g|1006,1007|1006,h|1946-07-07|1.8
+1007|-121|7|2.1|7.3|h|1947-08-08 01:01:01.111111111|h|h|h|h:h|1007,1008|1007,i|1947-08-08|2.1
+1008|-120|8|2.4|8.3|i|1948-09-09 01:01:01.111111111|i|i|i|i:i|1008,1009|1008,j|1948-09-09|2.4
+1009|-119|9|2.7|9.3|j|1949-10-10 01:01:01.111111111|j|j|j|j:j|1009,1010|1009,k|1949-10-10|2.7
+1010|-118|10|3.0|10.3|k|1950-11-11 01:01:01.111111111|k|k|k|k:k|1010,1011|1010,l|1950-11-11|3.0
+1011|-117|11|3.3|11.3|l|1951-12-12 01:01:01.111111111|l|l|l|l:l|1011,1012|1011,m|1951-12-12|3.3
+1012|-116|12|3.6|12.3|m|1952-01-13 01:01:01.111111111|m|m|m|m:m|1012,1013|1012,n|1952-01-13|3.6
+1013|-115|13|3.9|13.3|n|1953-02-14 01:01:01.111111111|n|n|n|n:n|1013,1014|1013,o|1953-02-14|3.9
+1014|-114|14|4.2|14.3|o|1954-03-15 01:01:01.111111111|o|o|o|o:o|1014,1015|1014,p|1954-03-15|4.2
+1015|-113|15|4.5|15.3|p|1955-04-16 01:01:01.111111111|p|p|p|p:p|1015,1016|1015,q|1955-04-16|4.5
+1016|-112|16|4.8|16.3|q|1956-05-17 01:01:01.111111111|q|q|q|q:q|1016,1017|1016,r|1956-05-17|4.8
+1017|-111|17|5.1|17.3|r|1957-06-18 01:01:01.111111111|r|r|r|r:r|1017,1018|1017,s|1957-06-18|5.1
+1018|-110|18|5.4|18.3|s|1958-07-19 01:01:01.111111111|s|s|s|s:s|1018,1019|1018,t|1958-07-19|5.4
+1019|-109|19|5.7|19.3|t|1959-08-20 01:01:01.111111111|t|t|t|t:t|1019,1020|1019,u|1959-08-20|5.7
+1020|-108|20|6.0|20.3|u|1960-09-21 01:01:01.111111111|u|u|u|u:u|1020,1021|1020,v|1960-09-21|6.0
+1021|-107|21|6.3|21.3|v|1961-10-22 01:01:01.111111111|v|v|v|v:v|1021,1022|1021,w|1961-10-22|6.3
+1022|-106|22|6.6|22.3|w|1962-11-23 01:01:01.111111111|w|w|w|w:w|1022,1023|1022,x|1962-11-23|6.6
+1023|-105|23|6.9|23.3|x|1963-12-24 01:01:01.111111111|x|x|x|x:x|1023,1024|1023,y|1963-12-24|6.9
+1024|-104|24|7.2|24.3|y|1964-01-25 01:01:01.111111111|y|y|y|y:y|1024,1025|1024,z|1964-01-25|7.2
+1025|-103|25|7.5|25.3|z|1965-02-26 01:01:01.111111111|z|z|z|z:z|1025,1026|1025,ba|1965-02-26|7.5
+1026|-102|26|7.8|26.3|ba|1966-03-27 01:01:01.111111111|ba|ba|ba|ba:ba|1026,1027|1026,bb|1966-03-27|7.8
+1027|-101|27|8.1|27.3|bb|1967-04-01 01:01:01.111111111|bb|bb|bb|bb:bb|1027,1028|1027,bc|1967-04-01|8.1
+1028|-100|28|8.4|28.3|bc|1968-05-02 01:01:01.111111111|bc|bc|bc|bc:bc|1028,1029|1028,bd|1968-05-02|8.4
+1029|-99|29|8.7|29.3|bd|1969-06-03 01:01:01.111111111|bd|bd|bd|bd:bd|1029,1030|1029,be|1969-06-03|8.7
+1030|-98|30|9.0|30.3|be|1970-07-04 01:01:01.111111111|be|be|be|be:be|1030,1031|1030,bf|1970-07-04|9.0
+1031|-97|31|9.3|31.3|bf|1971-08-05 01:01:01.111111111|bf|bf|bf|bf:bf|1031,1032|1031,bg|1971-08-05|9.3
+1032|-96|32|9.6|32.3|bg|1972-09-06 01:01:01.111111111|bg|bg|bg|bg:bg|1032,1033|1032,bh|1972-09-06|9.6
+1033|-95|33|9.9|33.3|bh|1973-10-07 01:01:01.111111111|bh|bh|bh|bh:bh|1033,1034|1033,bi|1973-10-07|9.9
+1034|-94|34|10.2|34.3|bi|1974-11-08 01:01:01.111111111|bi|bi|bi|bi:bi|1034,1035|1034,bj|1974-11-08|10.2
+1035|-93|35|10.5|35.3|bj|1975-12-09 01:01:01.111111111|bj|bj|bj|bj:bj|1035,1036|1035,bk|1975-12-09|10.5
+1036|-92|36|10.8|36.3|bk|1976-01-10 01:01:01.111111111|bk|bk|bk|bk:bk|1036,1037|1036,bl|1976-01-10|10.8
+1037|-91|37|11.1|37.3|bl|1977-02-11 01:01:01.111111111|bl|bl|bl|bl:bl|1037,1038|1037,bm|1977-02-11|11.1
+1038|-90|38|11.4|38.3|bm|1978-03-12 01:01:01.111111111|bm|bm|bm|bm:bm|1038,1039|1038,bn|1978-03-12|11.4
+1039|-89|39|11.7|39.3|bn|1979-04-13 01:01:01.111111111|bn|bn|bn|bn:bn|1039,1040|1039,bo|1979-04-13|11.7
+1040|-88|40|12.0|40.3|bo|1980-05-14 01:01:01.111111111|bo|bo|bo|bo:bo|1040,1041|1040,bp|1980-05-14|12.0
+1041|-87|41|12.3|41.3|bp|1981-06-15 01:01:01.111111111|bp|bp|bp|bp:bp|1041,1042|1041,bq|1981-06-15|12.3
+1042|-86|42|12.6|42.3|bq|1982-07-16 01:01:01.111111111|bq|bq|bq|bq:bq|1042,1043|1042,br|1982-07-16|12.6
+1043|-85|43|12.9|43.3|br|1983-08-17 01:01:01.111111111|br|br|br|br:br|1043,1044|1043,bs|1983-08-17|12.9
+1044|-84|44|13.2|44.3|bs|1984-09-18 01:01:01.111111111|bs|bs|bs|bs:bs|1044,1045|1044,bt|1984-09-18|13.2
+1045|-83|45|13.5|45.3|bt|1985-10-19 01:01:01.111111111|bt|bt|bt|bt:bt|1045,1046|1045,bu|1985-10-19|13.5
+1046|-82|46|13.8|46.3|bu|1986-11-20 01:01:01.111111111|bu|bu|bu|bu:bu|1046,1047|1046,bv|1986-11-20|13.8
+1047|-81|47|14.1|47.3|bv|1987-12-21 01:01:01.111111111|bv|bv|bv|bv:bv|1047,1048|1047,bw|1987-12-21|14.1
+1048|-80|48|14.4|48.3|bw|1988-01-22 01:01:01.111111111|bw|bw|bw|bw:bw|1048,1049|1048,bx|1988-01-22|14.4
+1049|-79|49|14.7|49.3|bx|1989-02-23 01:01:01.111111111|bx|bx|bx|bx:bx|1049,1050|1049,by|1989-02-23|14.7
+1050|-78|50|15.0|50.3|by|1990-03-24 01:01:01.111111111|by|by|by|by:by|1050,1051|1050,bz|1990-03-24|15.0
+1051|-77|51|15.3|51.3|bz|1991-04-25 01:01:01.111111111|bz|bz|bz|bz:bz|1051,1052|1051,ca|1991-04-25|15.3
+1052|-76|52|15.6|52.3|ca|1992-05-26 01:01:01.111111111|ca|ca|ca|ca:ca|1052,1053|1052,cb|1992-05-26|15.6
+1053|-75|53|15.9|53.3|cb|1993-06-27 01:01:01.111111111|cb|cb|cb|cb:cb|1053,1054|1053,cc|1993-06-27|15.9
+1054|-74|54|16.2|54.3|cc|1994-07-01 01:01:01.111111111|cc|cc|cc|cc:cc|1054,1055|1054,cd|1994-07-01|16.2
+1055|-73|55|16.5|55.3|cd|1995-08-02 01:01:01.111111111|cd|cd|cd|cd:cd|1055,1056|1055,ce|1995-08-02|16.5
+1056|-72|56|16.8|56.3|ce|1996-09-03 01:01:01.111111111|ce|ce|ce|ce:ce|1056,1057|1056,cf|1996-09-03|16.8
+1057|-71|57|17.1|57.3|cf|1997-10-04 01:01:01.111111111|cf|cf|cf|cf:cf|1057,1058|1057,cg|1997-10-04|17.1
+1058|-70|58|17.4|58.3|cg|1998-11-05 01:01:01.111111111|cg|cg|cg|cg:cg|1058,1059|1058,ch|1998-11-05|17.4
+1059|-69|59|17.7|59.3|ch|1999-12-06 01:01:01.111111111|ch|ch|ch|ch:ch|1059,1060|1059,ci|1999-12-06|17.7
+1060|-68|60|18.0|60.3|ci|2000-01-07 01:01:01.111111111|ci|ci|ci|ci:ci|1060,1061|1060,cj|2000-01-07|18.0
+1061|-67|61|18.3|61.3|cj|2001-02-08 01:01:01.111111111|cj|cj|cj|cj:cj|1061,1062|1061,ck|2001-02-08|18.3
+1062|-66|62|18.6|62.3|ck|2002-03-09 01:01:01.111111111|ck|ck|ck|ck:ck|1062,1063|1062,cl|2002-03-09|18.6
+1063|-65|63|18.9|63.3|cl|2003-04-10 01:01:01.111111111|cl|cl|cl|cl:cl|1063,1064|1063,cm|2003-04-10|18.9
+1064|-64|64|19.2|64.3|cm|2004-05-11 01:01:01.111111111|cm|cm|cm|cm:cm|1064,1065|1064,cn|2004-05-11|19.2
+1065|-63|65|19.5|65.3|cn|2005-06-12 01:01:01.111111111|cn|cn|cn|cn:cn|1065,1066|1065,co|2005-06-12|19.5
+1066|-62|66|19.8|66.3|co|2006-07-13 01:01:01.111111111|co|co|co|co:co|1066,1067|1066,cp|2006-07-13|19.8
+1067|-61|67|20.1|67.3|cp|2007-08-14 01:01:01.111111111|cp|cp|cp|cp:cp|1067,1068|1067,cq|2007-08-14|20.1
+1068|-60|68|20.4|68.3|cq|2008-09-15 01:01:01.111111111|cq|cq|cq|cq:cq|1068,1069|1068,cr|2008-09-15|20.4
+1069|-59|69|20.7|69.3|cr|2009-10-16 01:01:01.111111111|cr|cr|cr|cr:cr|1069,1070|1069,cs|2009-10-16|20.7
+1070|-58|70|21.0|70.3|cs|2010-11-17 01:01:01.111111111|cs|cs|cs|cs:cs|1070,1071|1070,ct|2010-11-17|21.0
+1071|-57|71|21.3|71.3|ct|2011-12-18 01:01:01.111111111|ct|ct|ct|ct:ct|1071,1072|1071,cu|2011-12-18|21.3
+1072|-56|72|21.6|72.3|cu|2012-01-19 01:01:01.111111111|cu|cu|cu|cu:cu|1072,1073|1072,cv|2012-01-19|21.6
+1073|-55|73|21.9|73.3|cv|2013-02-20 01:01:01.111111111|cv|cv|cv|cv:cv|1073,1074|1073,cw|2013-02-20|21.9
+1074|-54|74|22.2|74.3|cw|2014-03-21 01:01:01.111111111|cw|cw|cw|cw:cw|1074,1075|1074,cx|2014-03-21|22.2
+1075|-53|75|22.5|75.3|cx|2015-04-22 01:01:01.111111111|cx|cx|cx|cx:cx|1075,1076|1075,cy|2015-04-22|22.5
+1076|-52|76|22.8|76.3|cy|2016-05-23 01:01:01.111111111|cy|cy|cy|cy:cy|1076,1077|1076,cz|2016-05-23|22.8
+1077|-51|77|23.1|77.3|cz|2017-06-24 01:01:01.111111111|cz|cz|cz|cz:cz|1077,1078|1077,da|2017-06-24|23.1
+1078|-50|78|23.4|78.3|da|2018-07-25 01:01:01.111111111|da|da|da|da:da|1078,1079|1078,db|2018-07-25|23.4
+1079|-49|79|23.7|79.3|db|2019-08-26 01:01:01.111111111|db|db|db|db:db|1079,1080|1079,dc|2019-08-26|23.7
+1080|-48|80|24.0|80.3|dc|2020-09-27 01:01:01.111111111|dc|dc|dc|dc:dc|1080,1081|1080,dd|2020-09-27|24.0
+1081|-47|81|24.3|81.3|dd|2021-10-01 01:01:01.111111111|dd|dd|dd|dd:dd|1081,1082|1081,de|2021-10-01|24.3
+1082|-46|82|24.6|82.3|de|2022-11-02 01:01:01.111111111|de|de|de|de:de|1082,1083|1082,df|2022-11-02|24.6
+1083|-45|83|24.9|83.3|df|2023-12-03 01:01:01.111111111|df|df|df|df:df|1083,1084|1083,dg|2023-12-03|24.9
+1084|-44|84|25.2|84.3|dg|2024-01-04 01:01:01.111111111|dg|dg|dg|dg:dg|1084,1085|1084,dh|2024-01-04|25.2
+1085|-43|85|25.5|85.3|dh|2025-02-05 01:01:01.111111111|dh|dh|dh|dh:dh|1085,1086|1085,di|2025-02-05|25.5
+1086|-42|86|25.8|86.3|di|2026-03-06 01:01:01.111111111|di|di|di|di:di|1086,1087|1086,dj|2026-03-06|25.8
+1087|-41|87|26.1|87.3|dj|2027-04-07 01:01:01.111111111|dj|dj|dj|dj:dj|1087,1088|1087,dk|2027-04-07|26.1
+1088|-40|88|26.4|88.3|dk|2028-05-08 01:01:01.111111111|dk|dk|dk|dk:dk|1088,1089|1088,dl|2028-05-08|26.4
+1089|-39|89|26.7|89.3|dl|2029-06-09 01:01:01.111111111|dl|dl|dl|dl:dl|1089,1090|1089,dm|2029-06-09|26.7
+1090|-38|90|27.0|90.3|dm|2030-07-10 01:01:01.111111111|dm|dm|dm|dm:dm|1090,1091|1090,dn|2030-07-10|27.0
+1091|-37|91|27.3|91.3|dn|2031-08-11 01:01:01.111111111|dn|dn|dn|dn:dn|1091,1092|1091,do|2031-08-11|27.3
+1092|-36|92|27.6|92.3|do|2032-09-12 01:01:01.111111111|do|do|do|do:do|1092,1093|1092,dp|2032-09-12|27.6
+1093|-35|93|27.9|93.3|dp|2033-10-13 01:01:01.111111111|dp|dp|dp|dp:dp|1093,1094|1093,dq|2033-10-13|27.9
+1094|-34|94|28.2|94.3|dq|2034-11-14 01:01:01.111111111|dq|dq|dq|dq:dq|1094,1095|1094,dr|2034-11-14|28.2
+1095|-33|95|28.5|95.3|dr|2035-12-15 01:01:01.111111111|dr|dr|dr|dr:dr|1095,1096|1095,ds|2035-12-15|28.5
+1096|-32|96|28.8|96.3|ds|2036-01-16 01:01:01.111111111|ds|ds|ds|ds:ds|1096,1097|1096,dt|2036-01-16|28.8
+1097|-31|97|29.1|97.3|dt|2037-02-17 01:01:01.111111111|dt|dt|dt|dt:dt|1097,1098|1097,du|2037-02-17|29.1
+1098|-30|98|29.4|98.3|du|2038-03-18 01:01:01.111111111|du|du|du|du:du|1098,1099|1098,dv|2038-03-18|29.4
+1099|-29|99|29.7|99.3|dv|2039-04-19 01:01:01.111111111|dv|dv|dv|dv:dv|1099,1100|1099,dw|2039-04-19|29.7
+1100|-28|100|30.0|100.3|dw|2040-05-20 01:01:01.111111111|dw|dw|dw|dw:dw|1100,1101|1100,dx|2040-05-20|30.0
+1101|-27|101|30.3|101.3|dx|2041-06-21 01:01:01.111111111|dx|dx|dx|dx:dx|1101,1102|1101,dy|2041-06-21|30.3
+1102|-26|102|30.6|102.3|dy|2042-07-22 01:01:01.111111111|dy|dy|dy|dy:dy|1102,1103|1102,dz|2042-07-22|30.6
+1103|-25|103|30.9|103.3|dz|2043-08-23 01:01:01.111111111|dz|dz|dz|dz:dz|1103,1104|1103,ea|2043-08-23|30.9
+1104|-24|104|31.2|104.3|ea|2044-09-24 01:01:01.111111111|ea|ea|ea|ea:ea|1104,1105|1104,eb|2044-09-24|31.2
+1105|-23|105|31.5|105.3|eb|2045-10-25 01:01:01.111111111|eb|eb|eb|eb:eb|1105,1106|1105,ec|2045-10-25|31.5
+1106|-22|106|31.8|106.3|ec|2046-11-26 01:01:01.111111111|ec|ec|ec|ec:ec|1106,1107|1106,ed|2046-11-26|31.8
+1107|-21|107|32.1|107.3|ed|2047-12-27 01:01:01.111111111|ed|ed|ed|ed:ed|1107,1108|1107,ee|2047-12-27|32.1
+1108|-20|108|32.4|108.3|ee|2048-01-01 01:01:01.111111111|ee|ee|ee|ee:ee|1108,1109|1108,ef|2048-01-01|32.4
+1109|-19|109|32.7|109.3|ef|2049-02-02 01:01:01.111111111|ef|ef|ef|ef:ef|1109,1110|1109,eg|2049-02-02|32.7
+1110|-18|110|33.0|110.3|eg|2050-03-03 01:01:01.111111111|eg|eg|eg|eg:eg|1110,1111|1110,eh|2050-03-03|33.0
+1111|-17|111|33.3|111.3|eh|2051-04-04 01:01:01.111111111|eh|eh|eh|eh:eh|1111,1112|1111,ei|2051-04-04|33.3
+1112|-16|112|33.6|112.3|ei|2052-05-05 01:01:01.111111111|ei|ei|ei|ei:ei|1112,1113|1112,ej|2052-05-05|33.6
+1113|-15|113|33.9|113.3|ej|2053-06-06 01:01:01.111111111|ej|ej|ej|ej:ej|1113,1114|1113,ek|2053-06-06|33.9
+1114|-14|114|34.2|114.3|ek|2054-07-07 01:01:01.111111111|ek|ek|ek|ek:ek|1114,1115|1114,el|2054-07-07|34.2
+1115|-13|115|34.5|115.3|el|2055-08-08 01:01:01.111111111|el|el|el|el:el|1115,1116|1115,em|2055-08-08|34.5
+1116|-12|116|34.8|116.3|em|2056-09-09 01:01:01.111111111|em|em|em|em:em|1116,1117|1116,en|2056-09-09|34.8
+1117|-11|117|35.1|117.3|en|2057-10-10 01:01:01.111111111|en|en|en|en:en|1117,1118|1117,eo|2057-10-10|35.1
+1118|-10|118|35.4|118.3|eo|2058-11-11 01:01:01.111111111|eo|eo|eo|eo:eo|1118,1119|1118,ep|2058-11-11|35.4
+1119|-9|119|35.7|119.3|ep|2059-12-12 01:01:01.111111111|ep|ep|ep|ep:ep|1119,1120|1119,eq|2059-12-12|35.7
+1120|-8|120|36.0|120.3|eq|2060-01-13 01:01:01.111111111|eq|eq|eq|eq:eq|1120,1121|1120,er|2060-01-13|36.0
+1121|-7|121|36.3|121.3|er|2061-02-14 01:01:01.111111111|er|er|er|er:er|1121,1122|1121,es|2061-02-14|36.3
+1122|-6|122|36.6|122.3|es|2062-03-15 01:01:01.111111111|es|es|es|es:es|1122,1123|1122,et|2062-03-15|36.6
+1123|-5|123|36.9|123.3|et|2063-04-16 01:01:01.111111111|et|et|et|et:et|1123,1124|1123,eu|2063-04-16|36.9
+1124|-4|124|37.2|124.3|eu|2064-05-17 01:01:01.111111111|eu|eu|eu|eu:eu|1124,1125|1124,ev|2064-05-17|37.2
+1125|-3|125|37.5|125.3|ev|2065-06-18 01:01:01.111111111|ev|ev|ev|ev:ev|1125,1126|1125,ew|2065-06-18|37.5
+1126|-2|126|37.8|126.3|ew|2066-07-19 01:01:01.111111111|ew|ew|ew|ew:ew|1126,1127|1126,ex|2066-07-19|37.8
+1127|-1|127|38.1|127.3|ex|2067-08-20 01:01:01.111111111|ex|ex|ex|ex:ex|1127,1128|1127,ey|2067-08-20|38.1
+1128|0|128|38.4|128.3|ey|2068-09-21 01:01:01.111111111|ey|ey|ey|ey:ey|1128,1129|1128,ez|2068-09-21|38.4
+1129|1|129|38.7|129.3|ez|2069-10-22 01:01:01.111111111|ez|ez|ez|ez:ez|1129,1130|1129,fa|2069-10-22|38.7
+1130|2|130|39.0|130.3|fa|2070-11-23 01:01:01.111111111|fa|fa|fa|fa:fa|1130,1131|1130,fb|2070-11-23|39.0
+1131|3|131|39.3|131.3|fb|2071-12-24 01:01:01.111111111|fb|fb|fb|fb:fb|1131,1132|1131,fc|2071-12-24|39.3
+1132|4|132|39.6|132.3|fc|2072-01-25 01:01:01.111111111|fc|fc|fc|fc:fc|1132,1133|1132,fd|2072-01-25|39.6
+1133|5|133|39.9|133.3|fd|2073-02-26 01:01:01.111111111|fd|fd|fd|fd:fd|1133,1134|1133,fe|2073-02-26|39.9
+1134|6|134|40.2|134.3|fe|2074-03-27 01:01:01.111111111|fe|fe|fe|fe:fe|1134,1135|1134,ff|2074-03-27|40.2
+1135|7|135|40.5|135.3|ff|2075-04-01 01:01:01.111111111|ff|ff|ff|ff:ff|1135,1136|1135,fg|2075-04-01|40.5
+1136|8|136|40.8|136.3|fg|2076-05-02 01:01:01.111111111|fg|fg|fg|fg:fg|1136,1137|1136,fh|2076-05-02|40.8
+1137|9|137|41.1|137.3|fh|2077-06-03 01:01:01.111111111|fh|fh|fh|fh:fh|1137,1138|1137,fi|2077-06-03|41.1
+1138|10|138|41.4|138.3|fi|2078-07-04 01:01:01.111111111|fi|fi|fi|fi:fi|1138,1139|1138,fj|2078-07-04|41.4
+1139|11|139|41.7|139.3|fj|2079-08-05 01:01:01.111111111|fj|fj|fj|fj:fj|1139,1140|1139,fk|2079-08-05|41.7
+1140|12|140|42.0|140.3|fk|2080-09-06 01:01:01.111111111|fk|fk|fk|fk:fk|1140,1141|1140,fl|2080-09-06|42.0
+1141|13|141|42.3|141.3|fl|2081-10-07 01:01:01.111111111|fl|fl|fl|fl:fl|1141,1142|1141,fm|2081-10-07|42.3
+1142|14|142|42.6|142.3|fm|2082-11-08 01:01:01.111111111|fm|fm|fm|fm:fm|1142,1143|1142,fn|2082-11-08|42.6
+1143|15|143|42.9|143.3|fn|2083-12-09 01:01:01.111111111|fn|fn|fn|fn:fn|1143,1144|1143,fo|2083-12-09|42.9
+1144|16|144|43.2|144.3|fo|2084-01-10 01:01:01.111111111|fo|fo|fo|fo:fo|1144,1145|1144,fp|2084-01-10|43.2
+1145|17|145|43.5|145.3|fp|2085-02-11 01:01:01.111111111|fp|fp|fp|fp:fp|1145,1146|1145,fq|2085-02-11|43.5
+1146|18|146|43.8|146.3|fq|2086-03-12 01:01:01.111111111|fq|fq|fq|fq:fq|1146,1147|1146,fr|2086-03-12|43.8
+1147|19|147|44.1|147.3|fr|2087-04-13 01:01:01.111111111|fr|fr|fr|fr:fr|1147,1148|1147,fs|2087-04-13|44.1
+1148|20|148|44.4|148.3|fs|2088-05-14 01:01:01.111111111|fs|fs|fs|fs:fs|1148,1149|1148,ft|2088-05-14|44.4
+1149|21|149|44.7|149.3|ft|2089-06-15 01:01:01.111111111|ft|ft|ft|ft:ft|1149,1150|1149,fu|2089-06-15|44.7
+1150|22|150|45.0|150.3|fu|2090-07-16 01:01:01.111111111|fu|fu|fu|fu:fu|1150,1151|1150,fv|2090-07-16|45.0
+1151|23|151|45.3|151.3|fv|2091-08-17 01:01:01.111111111|fv|fv|fv|fv:fv|1151,1152|1151,fw|2091-08-17|45.3
+1152|24|152|45.6|152.3|fw|2092-09-18 01:01:01.111111111|fw|fw|fw|fw:fw|1152,1153|1152,fx|2092-09-18|45.6
+1153|25|153|45.9|153.3|fx|2093-10-19 01:01:01.111111111|fx|fx|fx|fx:fx|1153,1154|1153,fy|2093-10-19|45.9
+1154|26|154|46.2|154.3|fy|2094-11-20 01:01:01.111111111|fy|fy|fy|fy:fy|1154,1155|1154,fz|2094-11-20|46.2
+1155|27|155|46.5|155.3|fz|2095-12-21 01:01:01.111111111|fz|fz|fz|fz:fz|1155,1156|1155,ga|2095-12-21|46.5
+1156|28|156|46.8|156.3|ga|2096-01-22 01:01:01.111111111|ga|ga|ga|ga:ga|1156,1157|1156,gb|2096-01-22|46.8
+1157|29|157|47.1|157.3|gb|2097-02-23 01:01:01.111111111|gb|gb|gb|gb:gb|1157,1158|1157,gc|2097-02-23|47.1
+1158|30|158|47.4|158.3|gc|2098-03-24 01:01:01.111111111|gc|gc|gc|gc:gc|1158,1159|1158,gd|2098-03-24|47.4
+1159|31|159|47.7|159.3|gd|2099-04-25 01:01:01.111111111|gd|gd|gd|gd:gd|1159,1160|1159,ge|2099-04-25|47.7
+1160|32|160|48.0|160.3|ge|2100-05-26 01:01:01.111111111|ge|ge|ge|ge:ge|1160,1161|1160,gf|2100-05-26|48.0
+1161|33|161|48.3|161.3|gf|2101-06-27 01:01:01.111111111|gf|gf|gf|gf:gf|1161,1162|1161,gg|2101-06-27|48.3
+1162|34|162|48.6|162.3|gg|2102-07-01 01:01:01.111111111|gg|gg|gg|gg:gg|1162,1163|1162,gh|2102-07-01|48.6
+1163|35|163|48.9|163.3|gh|2103-08-02 01:01:01.111111111|gh|gh|gh|gh:gh|1163,1164|1163,gi|2103-08-02|48.9
+1164|36|164|49.2|164.3|gi|2104-09-03 01:01:01.111111111|gi|gi|gi|gi:gi|1164,1165|1164,gj|2104-09-03|49.2
+1165|37|165|49.5|165.3|gj|2105-10-04 01:01:01.111111111|gj|gj|gj|gj:gj|1165,1166|1165,gk|2105-10-04|49.5
+1166|38|166|49.8|166.3|gk|2106-11-05 01:01:01.111111111|gk|gk|gk|gk:gk|1166,1167|1166,gl|2106-11-05|49.8
+1167|39|167|50.1|167.3|gl|2107-12-06 01:01:01.111111111|gl|gl|gl|gl:gl|1167,1168|1167,gm|2107-12-06|50.1
+1168|40|168|50.4|168.3|gm|2108-01-07 01:01:01.111111111|gm|gm|gm|gm:gm|1168,1169|1168,gn|2108-01-07|50.4
+1169|41|169|50.7|169.3|gn|2109-02-08 01:01:01.111111111|gn|gn|gn|gn:gn|1169,1170|1169,go|2109-02-08|50.7
+1170|42|170|51.0|170.3|go|2110-03-09 01:01:01.111111111|go|go|go|go:go|1170,1171|1170,gp|2110-03-09|51.0
+1171|43|171|51.3|171.3|gp|2111-04-10 01:01:01.111111111|gp|gp|gp|gp:gp|1171,1172|1171,gq|2111-04-10|51.3
+1172|44|172|51.6|172.3|gq|2112-05-11 01:01:01.111111111|gq|gq|gq|gq:gq|1172,1173|1172,gr|2112-05-11|51.6
+1173|45|173|51.9|173.3|gr|2113-06-12 01:01:01.111111111|gr|gr|gr|gr:gr|1173,1174|1173,gs|2113-06-12|51.9
+1174|46|174|52.2|174.3|gs|2114-07-13 01:01:01.111111111|gs|gs|gs|gs:gs|1174,1175|1174,gt|2114-07-13|52.2
+1175|47|175|52.5|175.3|gt|2115-08-14 01:01:01.111111111|gt|gt|gt|gt:gt|1175,1176|1175,gu|2115-08-14|52.5
+1176|48|176|52.8|176.3|gu|2116-09-15 01:01:01.111111111|gu|gu|gu|gu:gu|1176,1177|1176,gv|2116-09-15|52.8
+1177|49|177|53.1|177.3|gv|2117-10-16 01:01:01.111111111|gv|gv|gv|gv:gv|1177,1178|1177,gw|2117-10-16|53.1
+1178|50|178|53.4|178.3|gw|2118-11-17 01:01:01.111111111|gw|gw|gw|gw:gw|1178,1179|1178,gx|2118-11-17|53.4
+1179|51|179|53.7|179.3|gx|2119-12-18 01:01:01.111111111|gx|gx|gx|gx:gx|1179,1180|1179,gy|2119-12-18|53.7
+1180|52|180|54.0|180.3|gy|2120-01-19 01:01:01.111111111|gy|gy|gy|gy:gy|1180,1181|1180,gz|2120-01-19|54.0
+1181|53|181|54.3|181.3|gz|2121-02-20 01:01:01.111111111|gz|gz|gz|gz:gz|1181,1182|1181,ha|2121-02-20|54.3
+1182|54|182|54.6|182.3|ha|2122-03-21 01:01:01.111111111|ha|ha|ha|ha:ha|1182,1183|1182,hb|2122-03-21|54.6
+1183|55|183|54.9|183.3|hb|2123-04-22 01:01:01.111111111|hb|hb|hb|hb:hb|1183,1184|1183,hc|2123-04-22|54.9
+1184|56|184|55.2|184.3|hc|2124-05-23 01:01:01.111111111|hc|hc|hc|hc:hc|1184,1185|1184,hd|2124-05-23|55.2
+1185|57|185|55.5|185.3|hd|2125-06-24 01:01:01.111111111|hd|hd|hd|hd:hd|1185,1186|1185,he|2125-06-24|55.5
+1186|58|186|55.8|186.3|he|2126-07-25 01:01:01.111111111|he|he|he|he:he|1186,1187|1186,hf|2126-07-25|55.8
+1187|59|187|56.1|187.3|hf|2127-08-26 01:01:01.111111111|hf|hf|hf|hf:hf|1187,1188|1187,hg|2127-08-26|56.1
+1188|60|188|56.4|188.3|hg|2128-09-27 01:01:01.111111111|hg|hg|hg|hg:hg|1188,1189|1188,hh|2128-09-27|56.4
+1189|61|189|56.7|189.3|hh|2129-10-01 01:01:01.111111111|hh|hh|hh|hh:hh|1189,1190|1189,hi|2129-10-01|56.7
+1190|62|190|57.0|190.3|hi|2130-11-02 01:01:01.111111111|hi|hi|hi|hi:hi|1190,1191|1190,hj|2130-11-02|57.0
+1191|63|191|57.3|191.3|hj|2131-12-03 01:01:01.111111111|hj|hj|hj|hj:hj|1191,1192|1191,hk|2131-12-03|57.3
+1192|64|192|57.6|192.3|hk|2132-01-04 01:01:01.111111111|hk|hk|hk|hk:hk|1192,1193|1192,hl|2132-01-04|57.6
+1193|65|193|57.9|193.3|hl|2133-02-05 01:01:01.111111111|hl|hl|hl|hl:hl|1193,1194|1193,hm|2133-02-05|57.9
+1194|66|194|58.2|194.3|hm|2134-03-06 01:01:01.111111111|hm|hm|hm|hm:hm|1194,1195|1194,hn|2134-03-06|58.2
+1195|67|195|58.5|195.3|hn|2135-04-07 01:01:01.111111111|hn|hn|hn|hn:hn|1195,1196|1195,ho|2135-04-07|58.5
+1196|68|196|58.8|196.3|ho|2136-05-08 01:01:01.111111111|ho|ho|ho|ho:ho|1196,1197|1196,hp|2136-05-08|58.8
+1197|69|197|59.1|197.3|hp|2137-06-09 01:01:01.111111111|hp|hp|hp|hp:hp|1197,1198|1197,hq|2137-06-09|59.1
+1198|70|198|59.4|198.3|hq|2138-07-10 01:01:01.111111111|hq|hq|hq|hq:hq|1198,1199|1198,hr|2138-07-10|59.4
+1199|71|199|59.7|199.3|hr|2139-08-11 01:01:01.111111111|hr|hr|hr|hr:hr|1199,1200|1199,hs|2139-08-11|59.7
+1200|72|200|60.0|200.3|hs|2140-09-12 01:01:01.111111111|hs|hs|hs|hs:hs|1200,1201|1200,ht|2140-09-12|60.0
+1201|73|201|60.3|201.3|ht|2141-10-13 01:01:01.111111111|ht|ht|ht|ht:ht|1201,1202|1201,hu|2141-10-13|60.3
+1202|74|202|60.6|202.3|hu|2142-11-14 01:01:01.111111111|hu|hu|hu|hu:hu|1202,1203|1202,hv|2142-11-14|60.6
+1203|75|203|60.9|203.3|hv|2143-12-15 01:01:01.111111111|hv|hv|hv|hv:hv|1203,1204|1203,hw|2143-12-15|60.9
+1204|76|204|61.2|204.3|hw|2144-01-16 01:01:01.111111111|hw|hw|hw|hw:hw|1204,1205|1204,hx|2144-01-16|61.2
+1205|77|205|61.5|205.3|hx|2145-02-17 01:01:01.111111111|hx|hx|hx|hx:hx|1205,1206|1205,hy|2145-02-17|61.5
+1206|78|206|61.8|206.3|hy|2146-03-18 01:01:01.111111111|hy|hy|hy|hy:hy|1206,1207|1206,hz|2146-03-18|61.8
+1207|79|207|62.1|207.3|hz|2147-04-19 01:01:01.111111111|hz|hz|hz|hz:hz|1207,1208|1207,ia|2147-04-19|62.1
+1208|80|208|62.4|208.3|ia|2148-05-20 01:01:01.111111111|ia|ia|ia|ia:ia|1208,1209|1208,ib|2148-05-20|62.4
+1209|81|209|62.7|209.3|ib|2149-06-21 01:01:01.111111111|ib|ib|ib|ib:ib|1209,1210|1209,ic|2149-06-21|62.7
+1210|82|210|63.0|210.3|ic|2150-07-22 01:01:01.111111111|ic|ic|ic|ic:ic|1210,1211|1210,id|2150-07-22|63.0
+1211|83|211|63.3|211.3|id|2151-08-23 01:01:01.111111111|id|id|id|id:id|1211,1212|1211,ie|2151-08-23|63.3
+1212|84|212|63.6|212.3|ie|2152-09-24 01:01:01.111111111|ie|ie|ie|ie:ie|1212,1213|1212,if|2152-09-24|63.6
+1213|85|213|63.9|213.3|if|2153-10-25 01:01:01.111111111|if|if|if|if:if|1213,1214|1213,ig|2153-10-25|63.9
+1214|86|214|64.2|214.3|ig|2154-11-26 01:01:01.111111111|ig|ig|ig|ig:ig|1214,1215|1214,ih|2154-11-26|64.2
+1215|87|215|64.5|215.3|ih|2155-12-27 01:01:01.111111111|ih|ih|ih|ih:ih|1215,1216|1215,ii|2155-12-27|64.5
+1216|88|216|64.8|216.3|ii|2156-01-01 01:01:01.111111111|ii|ii|ii|ii:ii|1216,1217|1216,ij|2156-01-01|64.8
+1217|89|217|65.1|217.3|ij|2157-02-02 01:01:01.111111111|ij|ij|ij|ij:ij|1217,1218|1217,ik|2157-02-02|65.1
+1218|90|218|65.4|218.3|ik|2158-03-03 01:01:01.111111111|ik|ik|ik|ik:ik|1218,1219|1218,il|2158-03-03|65.4
+1219|91|219|65.7|219.3|il|2159-04-04 01:01:01.111111111|il|il|il|il:il|1219,1220|1219,im|2159-04-04|65.7
+1220|92|220|66.0|220.3|im|2160-05-05 01:01:01.111111111|im|im|im|im:im|1220,1221|1220,in|2160-05-05|66.0
+1221|93|221|66.3|221.3|in|2161-06-06 01:01:01.111111111|in|in|in|in:in|1221,1222|1221,io|2161-06-06|66.3
+1222|94|222|66.6|222.3|io|2162-07-07 01:01:01.111111111|io|io|io|io:io|1222,1223|1222,ip|2162-07-07|66.6
+1223|95|223|66.9|223.3|ip|2163-08-08 01:01:01.111111111|ip|ip|ip|ip:ip|1223,1224|1223,iq|2163-08-08|66.9
+1224|96|224|67.2|224.3|iq|2164-09-09 01:01:01.111111111|iq|iq|iq|iq:iq|1224,1225|1224,ir|2164-09-09|67.2
+1225|97|225|67.5|225.3|ir|2165-10-10 01:01:01.111111111|ir|ir|ir|ir:ir|1225,1226|1225,is|2165-10-10|67.5
+1226|98|226|67.8|226.3|is|2166-11-11 01:01:01.111111111|is|is|is|is:is|1226,1227|1226,it|2166-11-11|67.8
+1227|99|227|68.1|227.3|it|2167-12-12 01:01:01.111111111|it|it|it|it:it|1227,1228|1227,iu|2167-12-12|68.1
+1228|100|228|68.4|228.3|iu|2168-01-13 01:01:01.111111111|iu|iu|iu|iu:iu|1228,1229|1228,iv|2168-01-13|68.4
+1229|101|229|68.7|229.3|iv|2169-02-14 01:01:01.111111111|iv|iv|iv|iv:iv|1229,1230|1229,iw|2169-02-14|68.7
+1230|102|230|69.0|230.3|iw|2170-03-15 01:01:01.111111111|iw|iw|iw|iw:iw|1230,1231|1230,ix|2170-03-15|69.0
+1231|103|231|69.3|231.3|ix|2171-04-16 01:01:01.111111111|ix|ix|ix|ix:ix|1231,1232|1231,iy|2171-04-16|69.3
+1232|104|232|69.6|232.3|iy|2172-05-17 01:01:01.111111111|iy|iy|iy|iy:iy|1232,1233|1232,iz|2172-05-17|69.6
+1233|105|233|69.9|233.3|iz|2173-06-18 01:01:01.111111111|iz|iz|iz|iz:iz|1233,1234|1233,ja|2173-06-18|69.9
+1234|106|234|70.2|234.3|ja|2174-07-19 01:01:01.111111111|ja|ja|ja|ja:ja|1234,1235|1234,jb|2174-07-19|70.2
+1235|107|235|70.5|235.3|jb|2175-08-20 01:01:01.111111111|jb|jb|jb|jb:jb|1235,1236|1235,jc|2175-08-20|70.5
+1236|108|236|70.8|236.3|jc|2176-09-21 01:01:01.111111111|jc|jc|jc|jc:jc|1236,1237|1236,jd|2176-09-21|70.8
+1237|109|237|71.1|237.3|jd|2177-10-22 01:01:01.111111111|jd|jd|jd|jd:jd|1237,1238|1237,je|2177-10-22|71.1
+1238|110|238|71.4|238.3|je|2178-11-23 01:01:01.111111111|je|je|je|je:je|1238,1239|1238,jf|2178-11-23|71.4
+1239|111|239|71.7|239.3|jf|2179-12-24 01:01:01.111111111|jf|jf|jf|jf:jf|1239,1240|1239,jg|2179-12-24|71.7
+1240|112|240|72.0|240.3|jg|2180-01-25 01:01:01.111111111|jg|jg|jg|jg:jg|1240,1241|1240,jh|2180-01-25|72.0
+1241|113|241|72.3|241.3|jh|2181-02-26 01:01:01.111111111|jh|jh|jh|jh:jh|1241,1242|1241,ji|2181-02-26|72.3
+1242|114|242|72.6|242.3|ji|2182-03-27 01:01:01.111111111|ji|ji|ji|ji:ji|1242,1243|1242,jj|2182-03-27|72.6
+1243|115|243|72.9|243.3|jj|2183-04-01 01:01:01.111111111|jj|jj|jj|jj:jj|1243,1244|1243,jk|2183-04-01|72.9
+1244|116|244|73.2|244.3|jk|2184-05-02 01:01:01.111111111|jk|jk|jk|jk:jk|1244,1245|1244,jl|2184-05-02|73.2
+1245|117|245|73.5|245.3|jl|2185-06-03 01:01:01.111111111|jl|jl|jl|jl:jl|1245,1246|1245,jm|2185-06-03|73.5
+1246|118|246|73.8|246.3|jm|2186-07-04 01:01:01.111111111|jm|jm|jm|jm:jm|1246,1247|1246,jn|2186-07-04|73.8
+1247|119|247|74.1|247.3|jn|2187-08-05 01:01:01.111111111|jn|jn|jn|jn:jn|1247,1248|1247,jo|2187-08-05|74.1
+1248|120|248|74.4|248.3|jo|2188-09-06 01:01:01.111111111|jo|jo|jo|jo:jo|1248,1249|1248,jp|2188-09-06|74.4
+1249|121|249|74.7|249.3|jp|2189-10-07 01:01:01.111111111|jp|jp|jp|jp:jp|1249,1250|1249,jq|2189-10-07|74.7
+1250|122|250|75.0|250.3|jq|2190-11-08 01:01:01.111111111|jq|jq|jq|jq:jq|1250,1251|1250,jr|2190-11-08|75.0
+1251|123|251|75.3|251.3|jr|2191-12-09 01:01:01.111111111|jr|jr|jr|jr:jr|1251,1252|1251,js|2191-12-09|75.3
+1252|124|252|75.6|252.3|js|2192-01-10 01:01:01.111111111|js|js|js|js:js|1252,1253|1252,jt|2192-01-10|75.6
+1253|125|253|75.9|253.3|jt|2193-02-11 01:01:01.111111111|jt|jt|jt|jt:jt|1253,1254|1253,ju|2193-02-11|75.9
+1254|126|254|76.2|254.3|ju|2194-03-12 01:01:01.111111111|ju|ju|ju|ju:ju|1254,1255|1254,jv|2194-03-12|76.2
+1255|127|255|76.5|255.3|jv|2195-04-13 01:01:01.111111111|jv|jv|jv|jv:jv|1255,1256|1255,jw|2195-04-13|76.5
+1256|-128|256|76.8|256.3|jw|2196-05-14 01:01:01.111111111|jw|jw|jw|jw:jw|1256,1257|1256,jx|2196-05-14|76.8
+1257|-127|257|77.1|257.3|jx|2197-06-15 01:01:01.111111111|jx|jx|jx|jx:jx|1257,1258|1257,jy|2197-06-15|77.1
+1258|-126|258|77.4|258.3|jy|2198-07-16 01:01:01.111111111|jy|jy|jy|jy:jy|1258,1259|1258,jz|2198-07-16|77.4
+1259|-125|259|77.7|259.3|jz|2199-08-17 01:01:01.111111111|jz|jz|jz|jz:jz|1259,1260|1259,ka|2199-08-17|77.7
+1260|-124|260|78.0|260.3|ka|2200-09-18 01:01:01.111111111|ka|ka|ka|ka:ka|1260,1261|1260,kb|2200-09-18|78.0
+1261|-123|261|78.3|261.3|kb|2201-10-19 01:01:01.111111111|kb|kb|kb|kb:kb|1261,1262|1261,kc|2201-10-19|78.3
+1262|-122|262|78.6|262.3|kc|2202-11-20 01:01:01.111111111|kc|kc|kc|kc:kc|1262,1263|1262,kd|2202-11-20|78.6
+1263|-121|263|78.9|263.3|kd|2203-12-21 01:01:01.111111111|kd|kd|kd|kd:kd|1263,1264|1263,ke|2203-12-21|78.9
+1264|-120|264|79.2|264.3|ke|2204-01-22 01:01:01.111111111|ke|ke|ke|ke:ke|1264,1265|1264,kf|2204-01-22|79.2
+1265|-119|265|79.5|265.3|kf|2205-02-23 01:01:01.111111111|kf|kf|kf|kf:kf|1265,1266|1265,kg|2205-02-23|79.5
+1266|-118|266|79.8|266.3|kg|2206-03-24 01:01:01.111111111|kg|kg|kg|kg:kg|1266,1267|1266,kh|2206-03-24|79.8
+1267|-117|267|80.1|267.3|kh|2207-04-25 01:01:01.111111111|kh|kh|kh|kh:kh|1267,1268|1267,ki|2207-04-25|80.1
+1268|-116|268|80.4|268.3|ki|2208-05-26 01:01:01.111111111|ki|ki|ki|ki:ki|1268,1269|1268,kj|2208-05-26|80.4
+1269|-115|269|80.7|269.3|kj|2209-06-27 01:01:01.111111111|kj|kj|kj|kj:kj|1269,1270|1269,kk|2209-06-27|80.7
+1270|-114|270|81.0|270.3|kk|2210-07-01 01:01:01.111111111|kk|kk|kk|kk:kk|1270,1271|1270,kl|2210-07-01|81.0
+1271|-113|271|81.3|271.3|kl|2211-08-02 01:01:01.111111111|kl|kl|kl|kl:kl|1271,1272|1271,km|2211-08-02|81.3
+1272|-112|272|81.6|272.3|km|2212-09-03 01:01:01.111111111|km|km|km|km:km|1272,1273|1272,kn|2212-09-03|81.6
+1273|-111|273|81.9|273.3|kn|2213-10-04 01:01:01.111111111|kn|kn|kn|kn:kn|1273,1274|1273,ko|2213-10-04|81.9
+1274|-110|274|82.2|274.3|ko|2214-11-05 01:01:01.111111111|ko|ko|ko|ko:ko|1274,1275|1274,kp|2214-11-05|82.2
+1275|-109|275|82.5|275.3|kp|2215-12-06 01:01:01.111111111|kp|kp|kp|kp:kp|1275,1276|1275,kq|2215-12-06|82.5
+1276|-108|276|82.8|276.3|kq|2216-01-07 01:01:01.111111111|kq|kq|kq|kq:kq|1276,1277|1276,kr|2216-01-07|82.8
+1277|-107|277|83.1|277.3|kr|2217-02-08 01:01:01.111111111|kr|kr|kr|kr:kr|1277,1278|1277,ks|2217-02-08|83.1
+1278|-106|278|83.4|278.3|ks|2218-03-09 01:01:01.111111111|ks|ks|ks|ks:ks|1278,1279|1278,kt|2218-03-09|83.4
+1279|-105|279|83.7|279.3|kt|2219-04-10 01:01:01.111111111|kt|kt|kt|kt:kt|1279,1280|1279,ku|2219-04-10|83.7
+1280|-104|280|84.0|280.3|ku|2220-05-11 01:01:01.111111111|ku|ku|ku|ku:ku|1280,1281|1280,kv|2220-05-11|84.0
+1281|-103|281|84.3|281.3|kv|2221-06-12 01:01:01.111111111|kv|kv|kv|kv:kv|1281,1282|1281,kw|2221-06-12|84.3
+1282|-102|282|84.6|282.3|kw|2222-07-13 01:01:01.111111111|kw|kw|kw|kw:kw|1282,1283|1282,kx|2222-07-13|84.6
+1283|-101|283|84.9|283.3|kx|2223-08-14 01:01:01.111111111|kx|kx|kx|kx:kx|1283,1284|1283,ky|2223-08-14|84.9
+1284|-100|284|85.2|284.3|ky|2224-09-15 01:01:01.111111111|ky|ky|ky|ky:ky|1284,1285|1284,kz|2224-09-15|85.2
+1285|-99|285|85.5|285.3|kz|2225-10-16 01:01:01.111111111|kz|kz|kz|kz:kz|1285,1286|1285,la|2225-10-16|85.5
+1286|-98|286|85.8|286.3|la|2226-11-17 01:01:01.111111111|la|la|la|la:la|1286,1287|1286,lb|2226-11-17|85.8
+1287|-97|287|86.1|287.3|lb|2227-12-18 01:01:01.111111111|lb|lb|lb|lb:lb|1287,1288|1287,lc|2227-12-18|86.1
+1288|-96|288|86.4|288.3|lc|2228-01-19 01:01:01.111111111|lc|lc|lc|lc:lc|1288,1289|1288,ld|2228-01-19|86.4
+1289|-95|289|86.7|289.3|ld|2229-02-20 01:01:01.111111111|ld|ld|ld|ld:ld|1289,1290|1289,le|2229-02-20|86.7
+1290|-94|290|87.0|290.3|le|2230-03-21 01:01:01.111111111|le|le|le|le:le|1290,1291|1290,lf|2230-03-21|87.0
+1291|-93|291|87.3|291.3|lf|2231-04-22 01:01:01.111111111|lf|lf|lf|lf:lf|1291,1292|1291,lg|2231-04-22|87.3
+1292|-92|292|87.6|292.3|lg|2232-05-23 01:01:01.111111111|lg|lg|lg|lg:lg|1292,1293|1292,lh|2232-05-23|87.6
+1293|-91|293|87.9|293.3|lh|2233-06-24 01:01:01.111111111|lh|lh|lh|lh:lh|1293,1294|1293,li|2233-06-24|87.9
+1294|-90|294|88.2|294.3|li|2234-07-25 01:01:01.111111111|li|li|li|li:li|1294,1295|1294,lj|2234-07-25|88.2
+1295|-89|295|88.5|295.3|lj|2235-08-26 01:01:01.111111111|lj|lj|lj|lj:lj|1295,1296|1295,lk|2235-08-26|88.5
+1296|-88|296|88.8|296.3|lk|2236-09-27 01:01:01.111111111|lk|lk|lk|lk:lk|1296,1297|1296,ll|2236-09-27|88.8
+1297|-87|297|89.1|297.3|ll|2237-10-01 01:01:01.111111111|ll|ll|ll|ll:ll|1297,1298|1297,lm|2237-10-01|89.1
+1298|-86|298|89.4|298.3|lm|2238-11-02 01:01:01.111111111|lm|lm|lm|lm:lm|1298,1299|1298,ln|2238-11-02|89.4
+1299|-85|299|89.7|299.3|ln|2239-12-03 01:01:01.111111111|ln|ln|ln|ln:ln|1299,1300|1299,lo|2239-12-03|89.7
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetInputFormat.java
index 5b65e5c..f4fadbb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/MapredParquetInputFormat.java
@@ -14,6 +14,8 @@
 package org.apache.hadoop.hive.ql.io.parquet;
 
 import java.io.IOException;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -34,7 +36,8 @@ import org.apache.parquet.hadoop.ParquetInputFormat;
  * NOTE: With HIVE-9235 we removed "implements VectorizedParquetInputFormat" since all data types
  *       are not currently supported.  Removing the interface turns off vectorization.
  */
-public class MapredParquetInputFormat extends FileInputFormat<NullWritable, ArrayWritable> {
+public class MapredParquetInputFormat extends FileInputFormat<NullWritable, ArrayWritable>
+  implements VectorizedInputFormatInterface {
 
   private static final Logger LOG = LoggerFactory.getLogger(MapredParquetInputFormat.class);
 
@@ -48,7 +51,7 @@ public class MapredParquetInputFormat extends FileInputFormat<NullWritable, Arra
 
   protected MapredParquetInputFormat(final ParquetInputFormat<ArrayWritable> inputFormat) {
     this.realInput = inputFormat;
-    vectorizedSelf = new VectorizedParquetInputFormat(inputFormat);
+    vectorizedSelf = new VectorizedParquetInputFormat();
   }
 
   @SuppressWarnings({ "unchecked", "rawtypes" })
@@ -69,8 +72,7 @@ public class MapredParquetInputFormat extends FileInputFormat<NullWritable, Arra
         if (LOG.isDebugEnabled()) {
           LOG.debug("Using row-mode record reader");
         }
-        return (RecordReader<NullWritable, ArrayWritable>)
-          new ParquetRecordReaderWrapper(realInput, split, job, reporter);
+        return new ParquetRecordReaderWrapper(realInput, split, job, reporter);
       }
     } catch (final InterruptedException e) {
       throw new RuntimeException("Cannot create a RecordReaderWrapper", e);

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
new file mode 100644
index 0000000..167f9b6
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ParquetRecordReaderBase.java
@@ -0,0 +1,171 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.io.parquet;
+
+import com.google.common.base.Strings;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.io.parquet.read.DataWritableReadSupport;
+import org.apache.hadoop.hive.ql.io.parquet.read.ParquetFilterPredicateConverter;
+import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.SerDeStats;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.RowGroupFilter;
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetInputFormat;
+import org.apache.parquet.hadoop.ParquetInputSplit;
+import org.apache.parquet.hadoop.api.InitContext;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.MessageTypeParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ParquetRecordReaderBase {
+  public static final Logger LOG = LoggerFactory.getLogger(ParquetRecordReaderBase.class);
+
+  protected Path file;
+  protected ProjectionPusher projectionPusher;
+  protected boolean skipTimestampConversion = false;
+  protected SerDeStats serDeStats;
+  protected JobConf jobConf;
+
+  protected int schemaSize;
+  protected List<BlockMetaData> filtedBlocks;
+  protected ParquetFileReader reader;
+
+  /**
+   * gets a ParquetInputSplit corresponding to a split given by Hive
+   *
+   * @param oldSplit The split given by Hive
+   * @param conf The JobConf of the Hive job
+   * @return a ParquetInputSplit corresponding to the oldSplit
+   * @throws IOException if the config cannot be enhanced or if the footer cannot be read from the file
+   */
+  @SuppressWarnings("deprecation")
+  protected ParquetInputSplit getSplit(
+    final org.apache.hadoop.mapred.InputSplit oldSplit,
+    final JobConf conf
+  ) throws IOException {
+    ParquetInputSplit split;
+    if (oldSplit instanceof FileSplit) {
+      final Path finalPath = ((FileSplit) oldSplit).getPath();
+      jobConf = projectionPusher.pushProjectionsAndFilters(conf, finalPath.getParent());
+
+      // TODO enable MetadataFilter by using readFooter(Configuration configuration, Path file,
+      // MetadataFilter filter) API
+      final ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(jobConf, finalPath);
+      final List<BlockMetaData> blocks = parquetMetadata.getBlocks();
+      final FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
+
+      final ReadSupport.ReadContext
+        readContext = new DataWritableReadSupport().init(new InitContext(jobConf,
+        null, fileMetaData.getSchema()));
+
+      // Compute stats
+      for (BlockMetaData bmd : blocks) {
+        serDeStats.setRowCount(serDeStats.getRowCount() + bmd.getRowCount());
+        serDeStats.setRawDataSize(serDeStats.getRawDataSize() + bmd.getTotalByteSize());
+      }
+
+      schemaSize = MessageTypeParser.parseMessageType(readContext.getReadSupportMetadata()
+        .get(DataWritableReadSupport.HIVE_TABLE_AS_PARQUET_SCHEMA)).getFieldCount();
+      final List<BlockMetaData> splitGroup = new ArrayList<BlockMetaData>();
+      final long splitStart = ((FileSplit) oldSplit).getStart();
+      final long splitLength = ((FileSplit) oldSplit).getLength();
+      for (final BlockMetaData block : blocks) {
+        final long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset();
+        if (firstDataPage >= splitStart && firstDataPage < splitStart + splitLength) {
+          splitGroup.add(block);
+        }
+      }
+      if (splitGroup.isEmpty()) {
+        LOG.warn("Skipping split, could not find row group in: " + oldSplit);
+        return null;
+      }
+
+      FilterCompat.Filter filter = setFilter(jobConf, fileMetaData.getSchema());
+      if (filter != null) {
+        filtedBlocks = RowGroupFilter.filterRowGroups(filter, splitGroup, fileMetaData.getSchema());
+        if (filtedBlocks.isEmpty()) {
+          LOG.debug("All row groups are dropped due to filter predicates");
+          return null;
+        }
+
+        long droppedBlocks = splitGroup.size() - filtedBlocks.size();
+        if (droppedBlocks > 0) {
+          LOG.debug("Dropping " + droppedBlocks + " row groups that do not pass filter predicate");
+        }
+      } else {
+        filtedBlocks = splitGroup;
+      }
+
+      if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION)) {
+        skipTimestampConversion = !Strings.nullToEmpty(fileMetaData.getCreatedBy()).startsWith("parquet-mr");
+      }
+      split = new ParquetInputSplit(finalPath,
+        splitStart,
+        splitLength,
+        oldSplit.getLocations(),
+        filtedBlocks,
+        readContext.getRequestedSchema().toString(),
+        fileMetaData.getSchema().toString(),
+        fileMetaData.getKeyValueMetaData(),
+        readContext.getReadSupportMetadata());
+      return split;
+    } else {
+      throw new IllegalArgumentException("Unknown split type: " + oldSplit);
+    }
+  }
+
+  public FilterCompat.Filter setFilter(final JobConf conf, MessageType schema) {
+    SearchArgument sarg = ConvertAstToSearchArg.createFromConf(conf);
+    if (sarg == null) {
+      return null;
+    }
+
+    // Create the Parquet FilterPredicate without including columns that do not exist
+    // on the schema (such as partition columns).
+    FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
+    if (p != null) {
+      // Filter may have sensitive information. Do not send to debug.
+      LOG.debug("PARQUET predicate push down generated.");
+      ParquetInputFormat.setFilterPredicate(conf, p);
+      return FilterCompat.get(p);
+    } else {
+      // Filter may have sensitive information. Do not send to debug.
+      LOG.debug("No PARQUET predicate push down is generated.");
+      return null;
+    }
+  }
+
+  public List<BlockMetaData> getFiltedBlocks() {
+    return filtedBlocks;
+  }
+
+  public SerDeStats getStats() {
+    return serDeStats;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/VectorizedParquetInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/VectorizedParquetInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/VectorizedParquetInputFormat.java
index 2072533..322178a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/VectorizedParquetInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/VectorizedParquetInputFormat.java
@@ -15,147 +15,29 @@ package org.apache.hadoop.hive.ql.io.parquet;
 
 import java.io.IOException;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.exec.vector.VectorColumnAssign;
-import org.apache.hadoop.hive.ql.exec.vector.VectorColumnAssignFactory;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
+import org.apache.hadoop.hive.ql.io.parquet.vector.VectorizedParquetRecordReader;
+import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
-import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.plan.MapWork;
-import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.FileInputFormat;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.parquet.hadoop.ParquetInputFormat;
 
 /**
  * Vectorized input format for Parquet files
  */
-public class VectorizedParquetInputFormat extends FileInputFormat<NullWritable, VectorizedRowBatch>
-  implements VectorizedInputFormatInterface {
-
-  private static final Logger LOG = LoggerFactory.getLogger(VectorizedParquetInputFormat.class);
-
-  /**
-   * Vectorized record reader for vectorized Parquet input format
-   */
-  private static class VectorizedParquetRecordReader implements
-      RecordReader<NullWritable, VectorizedRowBatch> {
-    private static final Logger LOG = LoggerFactory.getLogger(VectorizedParquetRecordReader.class);
-
-    private final ParquetRecordReaderWrapper internalReader;
-      private VectorizedRowBatchCtx rbCtx;
-      private Object[] partitionValues;
-      private ArrayWritable internalValues;
-      private NullWritable internalKey;
-      private VectorColumnAssign[] assigners;
-
-    public VectorizedParquetRecordReader(
-        ParquetInputFormat<ArrayWritable> realInput,
-        FileSplit split,
-        JobConf conf, Reporter reporter) throws IOException, InterruptedException {
-      internalReader = new ParquetRecordReaderWrapper(
-        realInput,
-        split,
-        conf,
-        reporter);
-      rbCtx = Utilities.getVectorizedRowBatchCtx(conf);
-      int partitionColumnCount = rbCtx.getPartitionColumnCount();
-      if (partitionColumnCount > 0) {
-        partitionValues = new Object[partitionColumnCount];
-        rbCtx.getPartitionValues(rbCtx, conf, split, partitionValues);
-      }
-    }
-
-      @Override
-      public NullWritable createKey() {
-        internalKey = internalReader.createKey();
-        return NullWritable.get();
-      }
-
-      @Override
-      public VectorizedRowBatch createValue() {
-        VectorizedRowBatch outputBatch;
-        outputBatch = rbCtx.createVectorizedRowBatch();
-        internalValues = internalReader.createValue();
-        return outputBatch;
-      }
-
-      @Override
-      public long getPos() throws IOException {
-        return internalReader.getPos();
-      }
+public class VectorizedParquetInputFormat
+  extends FileInputFormat<NullWritable, VectorizedRowBatch> {
 
-      @Override
-      public void close() throws IOException {
-        internalReader.close();
-      }
-
-      @Override
-      public float getProgress() throws IOException {
-        return internalReader.getProgress();
-      }
-
-    @Override
-    public boolean next(NullWritable key, VectorizedRowBatch outputBatch)
-        throws IOException {
-      if (assigners != null) {
-        assert(outputBatch.numCols == assigners.length);
-      }
-      outputBatch.reset();
-      int maxSize = outputBatch.getMaxSize();
-      try {
-        while (outputBatch.size < maxSize) {
-          if (false == internalReader.next(internalKey, internalValues)) {
-            outputBatch.endOfFile = true;
-            break;
-          }
-          Writable[] writables = internalValues.get();
-
-          if (null == assigners) {
-            // Normally we'd build the assigners from the rbCtx.rowOI, but with Parquet
-            // we have a discrepancy between the metadata type (Eg. tinyint -> BYTE) and
-            // the writable value (IntWritable). see Parquet's ETypeConverter class.
-            assigners = VectorColumnAssignFactory.buildAssigners(outputBatch, writables);
-          }
-
-          for(int i = 0; i < writables.length; ++i) {
-            assigners[i].assignObjectValue(writables[i], outputBatch.size);
-          }
-          ++outputBatch.size;
-         }
-      } catch (HiveException e) {
-        throw new RuntimeException(e);
-      }
-      return outputBatch.size > 0;
-    }
+  public VectorizedParquetInputFormat() {
   }
 
-  private final ParquetInputFormat<ArrayWritable> realInput;
-
-  public VectorizedParquetInputFormat(ParquetInputFormat<ArrayWritable> realInput) {
-    this.realInput = realInput;
-  }
-
-  @SuppressWarnings("unchecked")
   @Override
   public RecordReader<NullWritable, VectorizedRowBatch> getRecordReader(
-      InputSplit split, JobConf conf, Reporter reporter) throws IOException {
-    try {
-      return (RecordReader<NullWritable, VectorizedRowBatch>)
-        new VectorizedParquetRecordReader(realInput, (FileSplit) split, conf, reporter);
-    } catch (final InterruptedException e) {
-      throw new RuntimeException("Cannot create a VectorizedParquetRecordReader", e);
-    }
+    InputSplit inputSplit,
+    JobConf jobConf,
+    Reporter reporter) throws IOException {
+    return new VectorizedParquetRecordReader(inputSplit, jobConf);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
index 8d8b0c5..16064b2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/DataWritableReadSupport.java
@@ -70,7 +70,7 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
    * @param columns comma separated list of columns
    * @return list with virtual columns removed
    */
-  private static List<String> getColumnNames(final String columns) {
+  public static List<String> getColumnNames(final String columns) {
     return (List<String>) VirtualColumn.
         removeVirtualColumns(StringUtils.getStringCollection(columns));
   }
@@ -82,7 +82,7 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
    * @param types Comma separated list of types
    * @return A list of TypeInfo objects.
    */
-  private static List<TypeInfo> getColumnTypes(final String types) {
+  public static List<TypeInfo> getColumnTypes(final String types) {
     return TypeInfoUtils.getTypeInfosFromTypeString(types);
   }
 
@@ -177,7 +177,7 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
    * @param colTypes List of column types.
    * @return A MessageType object of projected columns.
    */
-  private static MessageType getSchemaByName(MessageType schema, List<String> colNames, List<TypeInfo> colTypes) {
+  public static MessageType getSchemaByName(MessageType schema, List<String> colNames, List<TypeInfo> colTypes) {
     List<Type> projectedFields = getProjectedGroupFields(schema, colNames, colTypes);
     Type[] typesArray = projectedFields.toArray(new Type[0]);
 
@@ -195,7 +195,7 @@ public class DataWritableReadSupport extends ReadSupport<ArrayWritable> {
    * @param colIndexes List of column indexes.
    * @return A MessageType object of the column names found.
    */
-  private static MessageType getSchemaByIndex(MessageType schema, List<String> colNames, List<Integer> colIndexes) {
+  public static MessageType getSchemaByIndex(MessageType schema, List<String> colNames, List<Integer> colIndexes) {
     List<Type> schemaTypes = new ArrayList<Type>();
 
     for (Integer i : colIndexes) {

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
index d2e1b13..ac430a6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
@@ -14,24 +14,19 @@
 package org.apache.hadoop.hive.ql.io.parquet.read;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 
+import org.apache.hadoop.hive.ql.io.parquet.ParquetRecordReaderBase;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.io.StatsProvidingRecordReader;
 import org.apache.hadoop.hive.ql.io.parquet.ProjectionPusher;
-import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
@@ -39,25 +34,12 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 
-import org.apache.parquet.filter2.compat.FilterCompat;
-import org.apache.parquet.filter2.compat.RowGroupFilter;
-import org.apache.parquet.filter2.predicate.FilterPredicate;
-import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.ParquetInputFormat;
 import org.apache.parquet.hadoop.ParquetInputSplit;
-import org.apache.parquet.hadoop.api.InitContext;
-import org.apache.parquet.hadoop.api.ReadSupport.ReadContext;
-import org.apache.parquet.hadoop.metadata.BlockMetaData;
-import org.apache.parquet.hadoop.metadata.FileMetaData;
-import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.ContextUtil;
-import org.apache.parquet.schema.MessageType;
-import org.apache.parquet.schema.MessageTypeParser;
 
-import com.google.common.base.Strings;
-
-public class ParquetRecordReaderWrapper implements RecordReader<NullWritable, ArrayWritable>,
-        StatsProvidingRecordReader {
+public class ParquetRecordReaderWrapper extends ParquetRecordReaderBase
+  implements RecordReader<NullWritable, ArrayWritable>, StatsProvidingRecordReader {
   public static final Logger LOG = LoggerFactory.getLogger(ParquetRecordReaderWrapper.class);
 
   private final long splitLen; // for getPos()
@@ -68,12 +50,6 @@ public class ParquetRecordReaderWrapper implements RecordReader<NullWritable, Ar
   private ArrayWritable valueObj = null;
   private boolean firstRecord = false;
   private boolean eof = false;
-  private int schemaSize;
-  private boolean skipTimestampConversion = false;
-  private JobConf jobConf;
-  private final ProjectionPusher projectionPusher;
-  private List<BlockMetaData> filtedBlocks;
-  private final SerDeStats serDeStats;
 
   public ParquetRecordReaderWrapper(
       final ParquetInputFormat<ArrayWritable> newInputFormat,
@@ -137,27 +113,6 @@ public class ParquetRecordReaderWrapper implements RecordReader<NullWritable, Ar
     }
   }
 
-  public FilterCompat.Filter setFilter(final JobConf conf, MessageType schema) {
-    SearchArgument sarg = ConvertAstToSearchArg.createFromConf(conf);
-    if (sarg == null) {
-      return null;
-    }
-
-    // Create the Parquet FilterPredicate without including columns that do not exist
-    // on the shema (such as partition columns).
-    FilterPredicate p = ParquetFilterPredicateConverter.toFilterPredicate(sarg, schema);
-    if (p != null) {
-      // Filter may have sensitive information. Do not send to debug.
-      LOG.debug("PARQUET predicate push down generated.");
-      ParquetInputFormat.setFilterPredicate(conf, p);
-      return FilterCompat.get(p);
-    } else {
-      // Filter may have sensitive information. Do not send to debug.
-      LOG.debug("No PARQUET predicate push down is generated.");
-      return null;
-    }
-  }
-
   @Override
   public void close() throws IOException {
     if (realReader != null) {
@@ -227,94 +182,4 @@ public class ParquetRecordReaderWrapper implements RecordReader<NullWritable, Ar
       throw new IOException(e);
     }
   }
-
-  /**
-   * gets a ParquetInputSplit corresponding to a split given by Hive
-   *
-   * @param oldSplit The split given by Hive
-   * @param conf The JobConf of the Hive job
-   * @return a ParquetInputSplit corresponding to the oldSplit
-   * @throws IOException if the config cannot be enhanced or if the footer cannot be read from the file
-   */
-  @SuppressWarnings("deprecation")
-  protected ParquetInputSplit getSplit(
-      final InputSplit oldSplit,
-      final JobConf conf
-      ) throws IOException {
-    ParquetInputSplit split;
-    if (oldSplit instanceof FileSplit) {
-      final Path finalPath = ((FileSplit) oldSplit).getPath();
-      jobConf = projectionPusher.pushProjectionsAndFilters(conf, finalPath.getParent());
-
-      final ParquetMetadata parquetMetadata = ParquetFileReader.readFooter(jobConf, finalPath);
-      final List<BlockMetaData> blocks = parquetMetadata.getBlocks();
-      final FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
-
-      final ReadContext readContext = new DataWritableReadSupport().init(new InitContext(jobConf,
-          null, fileMetaData.getSchema()));
-
-      // Compute stats
-      for (BlockMetaData bmd : blocks) {
-        serDeStats.setRowCount(serDeStats.getRowCount() + bmd.getRowCount());
-        serDeStats.setRawDataSize(serDeStats.getRawDataSize() + bmd.getTotalByteSize());
-      }
-
-      schemaSize = MessageTypeParser.parseMessageType(readContext.getReadSupportMetadata()
-          .get(DataWritableReadSupport.HIVE_TABLE_AS_PARQUET_SCHEMA)).getFieldCount();
-      final List<BlockMetaData> splitGroup = new ArrayList<BlockMetaData>();
-      final long splitStart = ((FileSplit) oldSplit).getStart();
-      final long splitLength = ((FileSplit) oldSplit).getLength();
-      for (final BlockMetaData block : blocks) {
-        final long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset();
-        if (firstDataPage >= splitStart && firstDataPage < splitStart + splitLength) {
-          splitGroup.add(block);
-        }
-      }
-      if (splitGroup.isEmpty()) {
-        LOG.warn("Skipping split, could not find row group in: " + (FileSplit) oldSplit);
-        return null;
-      }
-
-      FilterCompat.Filter filter = setFilter(jobConf, fileMetaData.getSchema());
-      if (filter != null) {
-        filtedBlocks = RowGroupFilter.filterRowGroups(filter, splitGroup, fileMetaData.getSchema());
-        if (filtedBlocks.isEmpty()) {
-          LOG.debug("All row groups are dropped due to filter predicates");
-          return null;
-        }
-
-        long droppedBlocks = splitGroup.size() - filtedBlocks.size();
-        if (droppedBlocks > 0) {
-          LOG.debug("Dropping " + droppedBlocks + " row groups that do not pass filter predicate");
-        }
-      } else {
-        filtedBlocks = splitGroup;
-      }
-
-      if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_PARQUET_TIMESTAMP_SKIP_CONVERSION)) {
-        skipTimestampConversion = !Strings.nullToEmpty(fileMetaData.getCreatedBy()).startsWith("parquet-mr");
-      }
-      split = new ParquetInputSplit(finalPath,
-          splitStart,
-          splitLength,
-          ((FileSplit) oldSplit).getLocations(),
-          filtedBlocks,
-          readContext.getRequestedSchema().toString(),
-          fileMetaData.getSchema().toString(),
-          fileMetaData.getKeyValueMetaData(),
-          readContext.getReadSupportMetadata());
-      return split;
-    } else {
-      throw new IllegalArgumentException("Unknown split type: " + oldSplit);
-    }
-  }
-
-  public List<BlockMetaData> getFiltedBlocks() {
-    return filtedBlocks;
-  }
-
-    @Override
-    public SerDeStats getStats() {
-      return serDeStats;
-    }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
index aace48e..3fd75d2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/timestamp/NanoTimeUtils.java
@@ -49,7 +49,7 @@ public class NanoTimeUtils {
      return parquetLocalCalendar.get();
    }
 
-   private static Calendar getCalendar(boolean skipConversion) {
+   public static Calendar getCalendar(boolean skipConversion) {
      Calendar calendar = skipConversion ? getLocalCalendar() : getGMTCalendar();
      calendar.clear(); // Reset all fields before reusing this instance
      return calendar;

http://git-wip-us.apache.org/repos/asf/hive/blob/936df7a1/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedColumnReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedColumnReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedColumnReader.java
new file mode 100644
index 0000000..5a9c7f9
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/vector/VectorizedColumnReader.java
@@ -0,0 +1,571 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.io.parquet.vector;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTime;
+import org.apache.hadoop.hive.ql.io.parquet.timestamp.NanoTimeUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.sql.Timestamp;
+import java.util.Arrays;
+
+import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL;
+import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL;
+import static org.apache.parquet.column.ValuesType.VALUES;
+
+/**
+ * It's column level Parquet reader which is used to read a batch of records for a column,
+ * part of the code is referred from Apache Spark and Apache Parquet.
+ */
+public class VectorizedColumnReader {
+
+  private static final Logger LOG = LoggerFactory.getLogger(VectorizedColumnReader.class);
+
+  private boolean skipTimestampConversion = false;
+
+  /**
+   * Total number of values read.
+   */
+  private long valuesRead;
+
+  /**
+   * value that indicates the end of the current page. That is,
+   * if valuesRead == endOfPageValueCount, we are at the end of the page.
+   */
+  private long endOfPageValueCount;
+
+  /**
+   * The dictionary, if this column has dictionary encoding.
+   */
+  private final Dictionary dictionary;
+
+  /**
+   * If true, the current page is dictionary encoded.
+   */
+  private boolean isCurrentPageDictionaryEncoded;
+
+  /**
+   * Maximum definition level for this column.
+   */
+  private final int maxDefLevel;
+
+  private int definitionLevel;
+  private int repetitionLevel;
+
+  /**
+   * Repetition/Definition/Value readers.
+   */
+  private IntIterator repetitionLevelColumn;
+  private IntIterator definitionLevelColumn;
+  private ValuesReader dataColumn;
+
+  /**
+   * Total values in the current page.
+   */
+  private int pageValueCount;
+
+  private final PageReader pageReader;
+  private final ColumnDescriptor descriptor;
+  private final Type type;
+
+  public VectorizedColumnReader(
+    ColumnDescriptor descriptor,
+    PageReader pageReader,
+    boolean skipTimestampConversion,
+    Type type) throws IOException {
+    this.descriptor = descriptor;
+    this.type = type;
+    this.pageReader = pageReader;
+    this.maxDefLevel = descriptor.getMaxDefinitionLevel();
+    this.skipTimestampConversion = skipTimestampConversion;
+
+    DictionaryPage dictionaryPage = pageReader.readDictionaryPage();
+    if (dictionaryPage != null) {
+      try {
+        this.dictionary = dictionaryPage.getEncoding().initDictionary(descriptor, dictionaryPage);
+        this.isCurrentPageDictionaryEncoded = true;
+      } catch (IOException e) {
+        throw new IOException("could not decode the dictionary for " + descriptor, e);
+      }
+    } else {
+      this.dictionary = null;
+      this.isCurrentPageDictionaryEncoded = false;
+    }
+  }
+
+  void readBatch(
+    int total,
+    ColumnVector column,
+    TypeInfo columnType) throws IOException {
+
+    int rowId = 0;
+    while (total > 0) {
+      // Compute the number of values we want to read in this page.
+      int leftInPage = (int) (endOfPageValueCount - valuesRead);
+      if (leftInPage == 0) {
+        readPage();
+        leftInPage = (int) (endOfPageValueCount - valuesRead);
+      }
+
+      int num = Math.min(total, leftInPage);
+      if (isCurrentPageDictionaryEncoded) {
+        LongColumnVector dictionaryIds = new LongColumnVector();
+        // Read and decode dictionary ids.
+        readDictionaryIDs(num, dictionaryIds, rowId);
+        decodeDictionaryIds(rowId, num, column, dictionaryIds);
+      } else {
+        // assign values in vector
+        PrimitiveTypeInfo primitiveColumnType = (PrimitiveTypeInfo) columnType;
+        switch (primitiveColumnType.getPrimitiveCategory()) {
+        case INT:
+        case BYTE:
+        case SHORT:
+          readIntegers(num, (LongColumnVector) column, rowId);
+          break;
+        case DATE:
+        case INTERVAL_YEAR_MONTH:
+        case LONG:
+          readLongs(num, (LongColumnVector) column, rowId);
+          break;
+        case BOOLEAN:
+          readBooleans(num, (LongColumnVector) column, rowId);
+          break;
+        case DOUBLE:
+          readDoubles(num, (DoubleColumnVector) column, rowId);
+          break;
+        case BINARY:
+        case STRING:
+        case CHAR:
+        case VARCHAR:
+          readBinaries(num, (BytesColumnVector) column, rowId);
+          break;
+        case FLOAT:
+          readFloats(num, (DoubleColumnVector) column, rowId);
+          break;
+        case DECIMAL:
+          readDecimal(num, (DecimalColumnVector) column, rowId);
+          break;
+        case INTERVAL_DAY_TIME:
+        case TIMESTAMP:
+        default:
+          throw new IOException(
+            "Unsupported type category: " + primitiveColumnType.getPrimitiveCategory());
+        }
+      }
+      rowId += num;
+      total -= num;
+    }
+  }
+
+  private void readDictionaryIDs(
+    int total,
+    LongColumnVector c,
+    int rowId) throws IOException {
+    int left = total;
+    while (left > 0) {
+      readRepetitionAndDefinitionLevels();
+      if (definitionLevel >= maxDefLevel) {
+        c.vector[rowId] = dataColumn.readValueDictionaryId();
+        c.isNull[rowId] = false;
+        c.isRepeating = c.isRepeating && (c.vector[0] == c.vector[rowId]);
+      } else {
+        c.isNull[rowId] = true;
+        c.isRepeating = false;
+        c.noNulls = false;
+      }
+      rowId++;
+      left--;
+    }
+  }
+
+  private void readIntegers(
+    int total,
+    LongColumnVector c,
+    int rowId) throws IOException {
+    int left = total;
+    while (left > 0) {
+      readRepetitionAndDefinitionLevels();
+      if (definitionLevel >= maxDefLevel) {
+        c.vector[rowId] = dataColumn.readInteger();
+        c.isNull[rowId] = false;
+        c.isRepeating = c.isRepeating && (c.vector[0] == c.vector[rowId]);
+      } else {
+        c.isNull[rowId] = true;
+        c.isRepeating = false;
+        c.noNulls = false;
+      }
+      rowId++;
+      left--;
+    }
+  }
+
+  private void readDoubles(
+    int total,
+    DoubleColumnVector c,
+    int rowId) throws IOException {
+    int left = total;
+    while (left > 0) {
+      readRepetitionAndDefinitionLevels();
+      if (definitionLevel >= maxDefLevel) {
+        c.vector[rowId] = dataColumn.readDouble();
+        c.isNull[rowId] = false;
+        c.isRepeating = c.isRepeating && (c.vector[0] == c.vector[rowId]);
+      } else {
+        c.isNull[rowId] = true;
+        c.isRepeating = false;
+        c.noNulls = false;
+      }
+      rowId++;
+      left--;
+    }
+  }
+
+  private void readBooleans(
+    int total,
+    LongColumnVector c,
+    int rowId) throws IOException {
+    int left = total;
+    while (left > 0) {
+      readRepetitionAndDefinitionLevels();
+      if (definitionLevel >= maxDefLevel) {
+        c.vector[rowId] = dataColumn.readBoolean() ? 1 : 0;
+        c.isNull[rowId] = false;
+        c.isRepeating = c.isRepeating && (c.vector[0] == c.vector[rowId]);
+      } else {
+        c.isNull[rowId] = true;
+        c.isRepeating = false;
+        c.noNulls = false;
+      }
+      rowId++;
+      left--;
+    }
+  }
+
+  private void readLongs(
+    int total,
+    LongColumnVector c,
+    int rowId) throws IOException {
+    int left = total;
+    while (left > 0) {
+      readRepetitionAndDefinitionLevels();
+      if (definitionLevel >= maxDefLevel) {
+        c.vector[rowId] = dataColumn.readLong();
+        c.isNull[rowId] = false;
+        c.isRepeating = c.isRepeating && (c.vector[0] == c.vector[rowId]);
+      } else {
+        c.isNull[rowId] = true;
+        c.isRepeating = false;
+        c.noNulls = false;
+      }
+      rowId++;
+      left--;
+    }
+  }
+
+  private void readFloats(
+    int total,
+    DoubleColumnVector c,
+    int rowId) throws IOException {
+    int left = total;
+    while (left > 0) {
+      readRepetitionAndDefinitionLevels();
+      if (definitionLevel >= maxDefLevel) {
+        c.vector[rowId] = dataColumn.readFloat();
+        c.isNull[rowId] = false;
+        c.isRepeating = c.isRepeating && (c.vector[0] == c.vector[rowId]);
+      } else {
+        c.isNull[rowId] = true;
+        c.isRepeating = false;
+        c.noNulls = false;
+      }
+      rowId++;
+      left--;
+    }
+  }
+
+  private void readDecimal(
+    int total,
+    DecimalColumnVector c,
+    int rowId) throws IOException {
+    int left = total;
+    c.precision = (short) type.asPrimitiveType().getDecimalMetadata().getPrecision();
+    c.scale = (short) type.asPrimitiveType().getDecimalMetadata().getScale();
+    while (left > 0) {
+      readRepetitionAndDefinitionLevels();
+      if (definitionLevel >= maxDefLevel) {
+        c.vector[rowId].set(dataColumn.readBytes().getBytesUnsafe(), c.scale);
+        c.isNull[rowId] = false;
+        c.isRepeating = c.isRepeating && (c.vector[0] == c.vector[rowId]);
+      } else {
+        c.isNull[rowId] = true;
+        c.isRepeating = false;
+        c.noNulls = false;
+      }
+      rowId++;
+      left--;
+    }
+  }
+
+  private void readBinaries(
+    int total,
+    BytesColumnVector c,
+    int rowId) throws IOException {
+    int left = total;
+    while (left > 0) {
+      readRepetitionAndDefinitionLevels();
+      if (definitionLevel >= maxDefLevel) {
+        c.setVal(rowId, dataColumn.readBytes().getBytesUnsafe());
+        c.isNull[rowId] = false;
+        // TODO figure out a better way to set repeat for Binary type
+        c.isRepeating = false;
+      } else {
+        c.isNull[rowId] = true;
+        c.isRepeating = false;
+        c.noNulls = false;
+      }
+      rowId++;
+      left--;
+    }
+  }
+
+  /**
+   * Reads `num` values into column, decoding the values from `dictionaryIds` and `dictionary`.
+   */
+  private void decodeDictionaryIds(int rowId, int num, ColumnVector column,
+                                   LongColumnVector dictionaryIds) {
+    System.arraycopy(dictionaryIds.isNull, rowId, column.isNull, rowId, num);
+    if (column.noNulls) {
+      column.noNulls = dictionaryIds.noNulls;
+    }
+    column.isRepeating = column.isRepeating && dictionaryIds.isRepeating;
+
+    switch (descriptor.getType()) {
+    case INT32:
+      for (int i = rowId; i < rowId + num; ++i) {
+        ((LongColumnVector) column).vector[i] =
+          dictionary.decodeToInt((int) dictionaryIds.vector[i]);
+      }
+      break;
+    case INT64:
+      for (int i = rowId; i < rowId + num; ++i) {
+        ((LongColumnVector) column).vector[i] =
+          dictionary.decodeToLong((int) dictionaryIds.vector[i]);
+      }
+      break;
+    case FLOAT:
+      for (int i = rowId; i < rowId + num; ++i) {
+        ((DoubleColumnVector) column).vector[i] =
+          dictionary.decodeToFloat((int) dictionaryIds.vector[i]);
+      }
+      break;
+    case DOUBLE:
+      for (int i = rowId; i < rowId + num; ++i) {
+        ((DoubleColumnVector) column).vector[i] =
+          dictionary.decodeToDouble((int) dictionaryIds.vector[i]);
+      }
+      break;
+    case INT96:
+      for (int i = rowId; i < rowId + num; ++i) {
+        ByteBuffer buf = dictionary.decodeToBinary((int) dictionaryIds.vector[i]).toByteBuffer();
+        buf.order(ByteOrder.LITTLE_ENDIAN);
+        long timeOfDayNanos = buf.getLong();
+        int julianDay = buf.getInt();
+        NanoTime nt = new NanoTime(julianDay, timeOfDayNanos);
+        Timestamp ts = NanoTimeUtils.getTimestamp(nt, skipTimestampConversion);
+        ((TimestampColumnVector) column).set(i, ts);
+      }
+      break;
+    case BINARY:
+    case FIXED_LEN_BYTE_ARRAY:
+      for (int i = rowId; i < rowId + num; ++i) {
+        ((BytesColumnVector) column)
+          .setVal(i, dictionary.decodeToBinary((int) dictionaryIds.vector[i]).getBytesUnsafe());
+      }
+      break;
+    default:
+      throw new UnsupportedOperationException("Unsupported type: " + descriptor.getType());
+    }
+  }
+
+  private void readRepetitionAndDefinitionLevels() {
+    repetitionLevel = repetitionLevelColumn.nextInt();
+    definitionLevel = definitionLevelColumn.nextInt();
+    valuesRead++;
+  }
+
+  private void readPage() throws IOException {
+    DataPage page = pageReader.readPage();
+    // TODO: Why is this a visitor?
+    page.accept(new DataPage.Visitor<Void>() {
+      @Override
+      public Void visit(DataPageV1 dataPageV1) {
+        readPageV1(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public Void visit(DataPageV2 dataPageV2) {
+        readPageV2(dataPageV2);
+        return null;
+      }
+    });
+  }
+
+  private void initDataReader(Encoding dataEncoding, byte[] bytes, int offset, int valueCount) throws IOException {
+    this.pageValueCount = valueCount;
+    this.endOfPageValueCount = valuesRead + pageValueCount;
+    if (dataEncoding.usesDictionary()) {
+      this.dataColumn = null;
+      if (dictionary == null) {
+        throw new IOException(
+          "could not read page in col " + descriptor +
+            " as the dictionary was missing for encoding " + dataEncoding);
+      }
+      dataColumn = dataEncoding.getDictionaryBasedValuesReader(descriptor, VALUES, dictionary);
+      this.isCurrentPageDictionaryEncoded = true;
+    } else {
+      if (dataEncoding != Encoding.PLAIN) {
+        throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding);
+      }
+      dataColumn = dataEncoding.getValuesReader(descriptor, VALUES);
+      this.isCurrentPageDictionaryEncoded = false;
+    }
+
+    try {
+      dataColumn.initFromPage(pageValueCount, bytes, offset);
+    } catch (IOException e) {
+      throw new IOException("could not read page in col " + descriptor, e);
+    }
+  }
+
+  private void readPageV1(DataPageV1 page) {
+    ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL);
+    ValuesReader dlReader = page.getDlEncoding().getValuesReader(descriptor, DEFINITION_LEVEL);
+    this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader);
+    this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader);
+    try {
+      byte[] bytes = page.getBytes().toByteArray();
+      LOG.debug("page size " + bytes.length + " bytes and " + pageValueCount + " records");
+      LOG.debug("reading repetition levels at 0");
+      rlReader.initFromPage(pageValueCount, bytes, 0);
+      int next = rlReader.getNextOffset();
+      LOG.debug("reading definition levels at " + next);
+      dlReader.initFromPage(pageValueCount, bytes, next);
+      next = dlReader.getNextOffset();
+      LOG.debug("reading data at " + next);
+      initDataReader(page.getValueEncoding(), bytes, next, page.getValueCount());
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + page + " in col " + descriptor, e);
+    }
+  }
+
+  private void readPageV2(DataPageV2 page) {
+    this.pageValueCount = page.getValueCount();
+    this.repetitionLevelColumn = newRLEIterator(descriptor.getMaxRepetitionLevel(),
+      page.getRepetitionLevels());
+    this.definitionLevelColumn = newRLEIterator(descriptor.getMaxDefinitionLevel(), page.getDefinitionLevels());
+    try {
+      LOG.debug("page data size " + page.getData().size() + " bytes and " + pageValueCount + " records");
+      initDataReader(page.getDataEncoding(), page.getData().toByteArray(), 0, page.getValueCount());
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + page + " in col " + descriptor, e);
+    }
+  }
+
+  private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) {
+    try {
+      if (maxLevel == 0) {
+        return new NullIntIterator();
+      }
+      return new RLEIntIterator(
+        new RunLengthBitPackingHybridDecoder(
+          BytesUtils.getWidthFromMaxInt(maxLevel),
+          new ByteArrayInputStream(bytes.toByteArray())));
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read levels in page for col " + descriptor, e);
+    }
+  }
+
+  /**
+   * Utility classes to abstract over different way to read ints with different encodings.
+   * TODO: remove this layer of abstraction?
+   */
+  abstract static class IntIterator {
+    abstract int nextInt();
+  }
+
+  protected static final class ValuesReaderIntIterator extends IntIterator {
+    ValuesReader delegate;
+
+    public ValuesReaderIntIterator(ValuesReader delegate) {
+      this.delegate = delegate;
+    }
+
+    @Override
+    int nextInt() {
+      return delegate.readInteger();
+    }
+  }
+
+  protected static final class RLEIntIterator extends IntIterator {
+    RunLengthBitPackingHybridDecoder delegate;
+
+    public RLEIntIterator(RunLengthBitPackingHybridDecoder delegate) {
+      this.delegate = delegate;
+    }
+
+    @Override
+    int nextInt() {
+      try {
+        return delegate.readInt();
+      } catch (IOException e) {
+        throw new ParquetDecodingException(e);
+      }
+    }
+  }
+
+  protected static final class NullIntIterator extends IntIterator {
+    @Override
+    int nextInt() { return 0; }
+  }
+}


[19/35] hive git commit: HIVE-14089 : complex type support in LLAP IO is broken (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-14089 : complex type support in LLAP IO is broken (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/6efa869f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6efa869f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6efa869f

Branch: refs/heads/hive-14535
Commit: 6efa869f71ec1283dc3b4d09562f24a8bdcb0aa3
Parents: aebc906
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri Nov 18 10:55:48 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri Nov 18 11:06:49 2016 -0800

----------------------------------------------------------------------
 .../hive/llap/io/api/impl/LlapInputFormat.java  |   5 -
 .../llap/io/decode/OrcEncodedDataConsumer.java  | 197 ++--
 .../llap/io/encoded/OrcEncodedDataReader.java   |  89 +-
 .../hive/llap/io/metadata/OrcFileMetadata.java  |   6 +
 .../orc/impl/ConvertTreeReaderFactory.java      |   2 +-
 .../org/apache/orc/impl/TreeReaderFactory.java  |  90 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |   6 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   2 +-
 .../hive/ql/io/orc/encoded/EncodedReader.java   |   2 +-
 .../ql/io/orc/encoded/EncodedReaderImpl.java    |  80 +-
 .../orc/encoded/EncodedTreeReaderFactory.java   | 985 +++++++++++++++----
 .../hadoop/hive/ql/io/orc/encoded/Reader.java   |   6 +-
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |  41 +-
 .../queries/clientpositive/vector_complex_all.q |  52 +-
 .../llap/vector_complex_all.q.out               | 315 +++---
 .../llap/vector_complex_join.q.out              |   4 +-
 .../clientpositive/vector_complex_all.q.out     | 302 +++---
 .../common/io/encoded/EncodedColumnBatch.java   |  52 +-
 18 files changed, 1464 insertions(+), 772 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
index a27266f..e803125 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
@@ -115,11 +115,6 @@ public class LlapInputFormat implements InputFormat<NullWritable, VectorizedRowB
     }
     boolean isVectorized = Utilities.getUseVectorizedInputFileFormat(job);
 
-    // validate for supported types. Until we fix HIVE-14089 we need this check.
-    if (useLlapIo) {
-      useLlapIo = Utilities.checkLlapIOSupportedTypes(job);
-    }
-
     if (!useLlapIo) {
       LlapIoImpl.LOG.warn("Not using LLAP IO for an unsupported split: " + split);
       return sourceInputFormat.getRecordReader(split, job, reporter);

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
index 2fa68a9..8bd985c 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
@@ -6,9 +6,7 @@
  * 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
- *
+ * 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.
@@ -18,6 +16,7 @@
 package org.apache.hadoop.hive.llap.io.decode;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.hive.common.io.encoded.EncodedColumnBatch;
@@ -40,6 +39,8 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.orc.CompressionCodec;
+import org.apache.orc.impl.PositionProvider;
+import org.apache.orc.OrcProto.RowIndexEntry;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Consumer;
 import org.apache.hadoop.hive.ql.io.orc.encoded.EncodedTreeReaderFactory;
 import org.apache.hadoop.hive.ql.io.orc.encoded.EncodedTreeReaderFactory.SettableTreeReader;
@@ -49,12 +50,19 @@ import org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl;
 import org.apache.orc.OrcUtils;
 import org.apache.orc.TypeDescription;
 import org.apache.orc.impl.TreeReaderFactory;
+import org.apache.orc.impl.TreeReaderFactory.StructTreeReader;
+import org.apache.orc.impl.TreeReaderFactory.TreeReader;
 import org.apache.hadoop.hive.ql.io.orc.WriterImpl;
 import org.apache.orc.OrcProto;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 
 public class OrcEncodedDataConsumer
   extends EncodedDataConsumer<OrcBatchKey, OrcEncodedColumnBatch> {
+  public static final Logger LOG = LoggerFactory.getLogger(OrcEncodedDataConsumer.class);
   private TreeReaderFactory.TreeReader[] columnReaders;
+  private int[] columnMapping; // Mapping from columnReaders (by index) to columns in file schema.
   private int previousStripeIndex = -1;
   private OrcFileMetadata fileMetadata; // We assume one request is only for one file.
   private CompressionCodec codec;
@@ -84,58 +92,6 @@ public class OrcEncodedDataConsumer
     stripes[m.getStripeIx()] = m;
   }
 
-  private static ColumnVector createColumn(List<OrcProto.Type> types,
-      final int columnId, int batchSize) {
-    OrcProto.Type type = types.get(columnId);
-    switch (type.getKind()) {
-      case BOOLEAN:
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-      case DATE:
-        return new LongColumnVector(batchSize);
-      case FLOAT:
-      case DOUBLE:
-        return new DoubleColumnVector(batchSize);
-      case BINARY:
-      case STRING:
-      case CHAR:
-      case VARCHAR:
-        return new BytesColumnVector(batchSize);
-      case TIMESTAMP:
-        return new TimestampColumnVector(batchSize);
-      case DECIMAL:
-        return new DecimalColumnVector(batchSize, type.getPrecision(),
-            type.getScale());
-      case STRUCT: {
-        List<Integer> subtypeIdxs = type.getSubtypesList();
-        ColumnVector[] fieldVector = new ColumnVector[subtypeIdxs.size()];
-        for(int i=0; i < fieldVector.length; ++i) {
-          fieldVector[i] = createColumn(types, subtypeIdxs.get(i), batchSize);
-        }
-        return new StructColumnVector(batchSize, fieldVector);
-      }
-      case UNION: {
-        List<Integer> subtypeIdxs = type.getSubtypesList();
-        ColumnVector[] fieldVector = new ColumnVector[subtypeIdxs.size()];
-        for(int i=0; i < fieldVector.length; ++i) {
-          fieldVector[i] = createColumn(types, subtypeIdxs.get(i), batchSize);
-        }
-        return new UnionColumnVector(batchSize, fieldVector);
-      }
-      case LIST:
-        return new ListColumnVector(batchSize, createColumn(types, type.getSubtypes(0), batchSize));
-      case MAP:
-        return new MapColumnVector(batchSize,
-            createColumn(types, type.getSubtypes(0), batchSize),
-            createColumn(types, type.getSubtypes(1), batchSize));
-      default:
-        throw new IllegalArgumentException("LLAP does not support " +
-            type.getKind());
-    }
-  }
-
   @Override
   protected void decodeBatch(OrcEncodedColumnBatch batch,
       Consumer<ColumnVectorBatch> downstreamConsumer) {
@@ -157,14 +113,18 @@ public class OrcEncodedDataConsumer
       }
       int maxBatchesRG = (int) ((nonNullRowCount / VectorizedRowBatch.DEFAULT_SIZE) + 1);
       int batchSize = VectorizedRowBatch.DEFAULT_SIZE;
-      int numCols = batch.getColumnIxs().length;
+      TypeDescription schema = fileMetadata.getSchema();
+
       if (columnReaders == null || !sameStripe) {
-        this.columnReaders = EncodedTreeReaderFactory.createEncodedTreeReader(numCols,
-            fileMetadata.getTypes(), stripeMetadata.getEncodings(), batch, codec, skipCorrupt,
-                stripeMetadata.getWriterTimezone());
-        positionInStreams(columnReaders, batch, numCols, stripeMetadata);
+        int[] columnMapping = new int[schema.getChildren().size()];
+        StructTreeReader treeReader = EncodedTreeReaderFactory.createRootTreeReader(
+            schema, stripeMetadata.getEncodings(), batch, codec, skipCorrupt,
+            stripeMetadata.getWriterTimezone(), columnMapping);
+        this.columnReaders = treeReader.getChildReaders();
+        this.columnMapping = Arrays.copyOf(columnMapping, columnReaders.length);
+        positionInStreams(columnReaders, batch, stripeMetadata);
       } else {
-        repositionInStreams(this.columnReaders, batch, sameStripe, numCols, stripeMetadata);
+        repositionInStreams(this.columnReaders, batch, sameStripe, stripeMetadata);
       }
       previousStripeIndex = currentStripeIndex;
 
@@ -176,18 +136,17 @@ public class OrcEncodedDataConsumer
         }
 
         ColumnVectorBatch cvb = cvbPool.take();
-        assert cvb.cols.length == batch.getColumnIxs().length; // Must be constant per split.
+        // assert cvb.cols.length == batch.getColumnIxs().length; // Must be constant per split.
         cvb.size = batchSize;
-        List<OrcProto.Type> types = fileMetadata.getTypes();
-        int[] columnMapping = batch.getColumnIxs();
-        for (int idx = 0; idx < batch.getColumnIxs().length; idx++) {
+        for (int idx = 0; idx < columnReaders.length; ++idx) {
+          TreeReader reader = columnReaders[idx];
           if (cvb.cols[idx] == null) {
             // Orc store rows inside a root struct (hive writes it this way).
             // When we populate column vectors we skip over the root struct.
-            cvb.cols[idx] = createColumn(types, columnMapping[idx], batchSize);
+            cvb.cols[idx] = createColumn(schema.getChildren().get(columnMapping[idx]), batchSize);
           }
           cvb.cols[idx].ensureSize(batchSize, false);
-          columnReaders[idx].nextVector(cvb.cols[idx], null, batchSize);
+          reader.nextVector(cvb.cols[idx], null, batchSize);
         }
 
         // we are done reading a batch, send it to consumer for processing
@@ -203,36 +162,98 @@ public class OrcEncodedDataConsumer
     }
   }
 
+  private ColumnVector createColumn(TypeDescription type, int batchSize) {
+    switch (type.getCategory()) {
+      case BOOLEAN:
+      case BYTE:
+      case SHORT:
+      case INT:
+      case LONG:
+      case DATE:
+        return new LongColumnVector(batchSize);
+      case FLOAT:
+      case DOUBLE:
+        return new DoubleColumnVector(batchSize);
+      case BINARY:
+      case STRING:
+      case CHAR:
+      case VARCHAR:
+        return new BytesColumnVector(batchSize);
+      case TIMESTAMP:
+        return new TimestampColumnVector(batchSize);
+      case DECIMAL:
+        return new DecimalColumnVector(batchSize, type.getPrecision(),
+            type.getScale());
+      case STRUCT: {
+        List<TypeDescription> subtypeIdxs = type.getChildren();
+        ColumnVector[] fieldVector = new ColumnVector[subtypeIdxs.size()];
+        for(int i = 0; i < fieldVector.length; ++i) {
+          fieldVector[i] = createColumn(subtypeIdxs.get(i), batchSize);
+        }
+        return new StructColumnVector(batchSize, fieldVector);
+      }
+      case UNION: {
+        List<TypeDescription> subtypeIdxs = type.getChildren();
+        ColumnVector[] fieldVector = new ColumnVector[subtypeIdxs.size()];
+        for(int i=0; i < fieldVector.length; ++i) {
+          fieldVector[i] = createColumn(subtypeIdxs.get(i), batchSize);
+        }
+        return new UnionColumnVector(batchSize, fieldVector);
+      }
+      case LIST:
+        return new ListColumnVector(batchSize, createColumn(type.getChildren().get(0), batchSize));
+      case MAP:
+        List<TypeDescription> subtypeIdxs = type.getChildren();
+        return new MapColumnVector(batchSize, createColumn(subtypeIdxs.get(0), batchSize),
+            createColumn(subtypeIdxs.get(1), batchSize));
+      default:
+        throw new IllegalArgumentException("LLAP does not support " + type.getCategory());
+    }
+  }
+
   private void positionInStreams(TreeReaderFactory.TreeReader[] columnReaders,
-      EncodedColumnBatch<OrcBatchKey> batch, int numCols,
-      OrcStripeMetadata stripeMetadata) throws IOException {
-    for (int i = 0; i < numCols; i++) {
-      int columnIndex = batch.getColumnIxs()[i];
-      int rowGroupIndex = batch.getBatchKey().rgIx;
-      OrcProto.RowIndex rowIndex = stripeMetadata.getRowIndexes()[columnIndex];
-      OrcProto.RowIndexEntry rowIndexEntry = rowIndex.getEntry(rowGroupIndex);
-      columnReaders[i].seek(new RecordReaderImpl.PositionProviderImpl(rowIndexEntry));
+      EncodedColumnBatch<OrcBatchKey> batch, OrcStripeMetadata stripeMetadata) throws IOException {
+    PositionProvider[] pps = createPositionProviders(columnReaders, batch, stripeMetadata);
+    if (pps == null) return;
+    for (int i = 0; i < columnReaders.length; i++) {
+      columnReaders[i].seek(pps);
     }
   }
 
   private void repositionInStreams(TreeReaderFactory.TreeReader[] columnReaders,
-      EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe, int numCols,
+      EncodedColumnBatch<OrcBatchKey> batch, boolean sameStripe,
       OrcStripeMetadata stripeMetadata) throws IOException {
-    for (int i = 0; i < numCols; i++) {
-      int columnIndex = batch.getColumnIxs()[i];
-      int rowGroupIndex = batch.getBatchKey().rgIx;
-      ColumnStreamData[] streamBuffers = batch.getColumnData()[i];
-      OrcProto.RowIndex rowIndex = stripeMetadata.getRowIndexes()[columnIndex];
-      OrcProto.RowIndexEntry rowIndexEntry = rowIndex.getEntry(rowGroupIndex);
-      ((SettableTreeReader)columnReaders[i]).setBuffers(streamBuffers, sameStripe);
-      // TODO: When hive moves to java8, make updateTimezone() as default method in SettableTreeReader so that we can
-      // avoid this check
-      if (columnReaders[i] instanceof EncodedTreeReaderFactory.TimestampStreamReader && !sameStripe) {
-        ((EncodedTreeReaderFactory.TimestampStreamReader) columnReaders[i])
+    PositionProvider[] pps = createPositionProviders(columnReaders, batch, stripeMetadata);
+    if (pps == null) return;
+    for (int i = 0; i < columnReaders.length; i++) {
+      TreeReader reader = columnReaders[i];
+      ((SettableTreeReader) reader).setBuffers(batch, sameStripe);
+      // TODO: When hive moves to java8, make updateTimezone() as default method in
+      // SettableTreeReader so that we can avoid this check.
+      if (reader instanceof EncodedTreeReaderFactory.TimestampStreamReader && !sameStripe) {
+        ((EncodedTreeReaderFactory.TimestampStreamReader) reader)
                 .updateTimezone(stripeMetadata.getWriterTimezone());
       }
-      columnReaders[i].seek(new RecordReaderImpl.PositionProviderImpl(rowIndexEntry));
+      reader.seek(pps);
+    }
+  }
+
+  private PositionProvider[] createPositionProviders(TreeReaderFactory.TreeReader[] columnReaders,
+      EncodedColumnBatch<OrcBatchKey> batch, OrcStripeMetadata stripeMetadata) throws IOException {
+    if (columnReaders.length == 0) return null;
+    int rowGroupIndex = batch.getBatchKey().rgIx;
+    if (rowGroupIndex == OrcEncodedColumnBatch.ALL_RGS) {
+      throw new IOException("Cannot position readers without RG information");
+    }
+    // TODO: this assumes indexes in getRowIndexes would match column IDs
+    OrcProto.RowIndex[] ris = stripeMetadata.getRowIndexes();
+    PositionProvider[] pps = new PositionProvider[ris.length];
+    for (int i = 0; i < ris.length; ++i) {
+      OrcProto.RowIndex ri = ris[i];
+      if (ri == null) continue;
+      pps[i] = new RecordReaderImpl.PositionProviderImpl(ri.getEntry(rowGroupIndex));
     }
+    return pps;
   }
 
   private long getRowCount(OrcProto.RowIndexEntry rowIndexEntry) {

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
index 72f733a..39f4112 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
@@ -138,7 +138,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
   private final BufferUsageManager bufferManager;
   private final Configuration conf;
   private final FileSplit split;
-  private List<Integer> columnIds;
+  private List<Integer> includedColumnIds;
   private final SearchArgument sarg;
   private final String[] columnNames;
   private final OrcEncodedDataConsumer consumer;
@@ -174,9 +174,9 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
     this.bufferManager = bufferManager;
     this.conf = conf;
     this.split = split;
-    this.columnIds = columnIds;
-    if (this.columnIds != null) {
-      Collections.sort(this.columnIds);
+    this.includedColumnIds = columnIds;
+    if (this.includedColumnIds != null) {
+      Collections.sort(this.includedColumnIds);
     }
     this.sarg = sarg;
     this.columnNames = columnNames;
@@ -197,7 +197,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
     fileKey = determineFileId(fs, split,
         HiveConf.getBoolVar(conf, ConfVars.LLAP_CACHE_ALLOW_SYNTHETIC_FILEID));
     fileMetadata = getOrReadFileMetadata();
-    globalIncludes = OrcInputFormat.genIncludedColumns(fileMetadata.getTypes(), columnIds, true);
+    globalIncludes = OrcInputFormat.genIncludedColumns(fileMetadata.getTypes(), includedColumnIds, true);
     consumer.setFileMetadata(fileMetadata);
     consumer.setIncludedColumns(globalIncludes);
   }
@@ -242,8 +242,8 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
         + (fileKey == null ? "" : " (" + fileKey + ")"));
     try {
       validateFileMetadata();
-      if (columnIds == null) {
-        columnIds = createColumnIds(fileMetadata);
+      if (includedColumnIds == null) {
+        includedColumnIds = getAllColumnIds(fileMetadata);
       }
 
       // 2. Determine which stripes to read based on the split.
@@ -303,13 +303,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
       return null;
     }
 
-    // 4. Get data from high-level cache.
-    //    If some cols are fully in cache, this will also give us the modified list of columns to
-    //    read for every stripe (null means read all of them - the usual path). In any case,
-    //    readState will be modified for column x rgs that were fetched from high-level cache.
-    List<Integer>[] stripeColsToRead = null;
-
-    // 5. Create encoded data reader.
+    // 4. Create encoded data reader.
     try {
       ensureOrcReader();
       // Reader creating updates HDFS counters, don't do it here.
@@ -335,17 +329,17 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
       }
       int stripeIx = stripeIxFrom + stripeIxMod;
       boolean[][] colRgs = null;
-      boolean[] stripeIncludes = null;
       OrcStripeMetadata stripeMetadata = null;
       StripeInformation stripe;
       try {
-        List<Integer> cols = stripeColsToRead == null ? null : stripeColsToRead[stripeIxMod];
-        if (cols != null && cols.isEmpty()) continue; // No need to read this stripe.
         stripe = fileMetadata.getStripes().get(stripeIx);
 
         LlapIoImpl.ORC_LOGGER.trace("Reading stripe {}: {}, {}", stripeIx, stripe.getOffset(),
             stripe.getLength());
         colRgs = readState[stripeIxMod];
+        if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
+          LlapIoImpl.ORC_LOGGER.trace("readState[{}]: {}", stripeIxMod, Arrays.toString(colRgs));
+        }
         // We assume that NO_RGS value is only set from SARG filter and for all columns;
         // intermediate changes for individual columns will unset values in the array.
         // Skip this case for 0-column read. We could probably special-case it just like we do
@@ -353,17 +347,6 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
         if (colRgs.length > 0 && colRgs[0] ==
             RecordReaderImpl.SargApplier.READ_NO_RGS) continue;
 
-        // 6.1. Determine the columns to read (usually the same as requested).
-        if (cols == null || cols.size() == colRgs.length) {
-          cols = columnIds;
-          stripeIncludes = globalIncludes;
-        } else {
-          // We are reading subset of the original columns, remove unnecessary bitmasks/etc.
-          // This will never happen w/o high-level cache.
-          stripeIncludes = OrcInputFormat.genIncludedColumns(fileMetadata.getTypes(), cols, true);
-          colRgs = genStripeColRgs(cols, colRgs);
-        }
-
         // 6.2. Ensure we have stripe metadata. We might have read it before for RG filtering.
         boolean isFoundInCache = false;
         if (stripeMetadatas != null) {
@@ -379,27 +362,27 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
             ensureMetadataReader();
             long startTimeHdfs = counters.startTimeCounter();
             stripeMetadata = new OrcStripeMetadata(new OrcBatchKey(fileKey, stripeIx, 0),
-                metadataReader, stripe, stripeIncludes, sargColumns);
+                metadataReader, stripe, globalIncludes, sargColumns);
             counters.incrTimeCounter(LlapIOCounters.HDFS_TIME_NS, startTimeHdfs);
             if (hasFileId && metadataCache != null) {
               stripeMetadata = metadataCache.putStripeMetadata(stripeMetadata);
               if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
                 LlapIoImpl.ORC_LOGGER.trace("Caching stripe {} metadata with includes: {}",
-                    stripeKey.stripeIx, DebugUtils.toString(stripeIncludes));
+                    stripeKey.stripeIx, DebugUtils.toString(globalIncludes));
               }
             }
           }
           consumer.setStripeMetadata(stripeMetadata);
         }
-        if (!stripeMetadata.hasAllIndexes(stripeIncludes)) {
+        if (!stripeMetadata.hasAllIndexes(globalIncludes)) {
           if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
             LlapIoImpl.ORC_LOGGER.trace("Updating indexes in stripe {} metadata for includes: {}",
-                stripeKey.stripeIx, DebugUtils.toString(stripeIncludes));
+                stripeKey.stripeIx, DebugUtils.toString(globalIncludes));
           }
           assert isFoundInCache;
           counters.incrCounter(LlapIOCounters.METADATA_CACHE_MISS);
           ensureMetadataReader();
-          updateLoadedIndexes(stripeMetadata, stripe, stripeIncludes, sargColumns);
+          updateLoadedIndexes(stripeMetadata, stripe, globalIncludes, sargColumns);
         } else if (isFoundInCache) {
           counters.incrCounter(LlapIOCounters.METADATA_CACHE_HIT);
         }
@@ -415,7 +398,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
         return null;
       }
 
-      // 6.3. Finally, hand off to the stripe reader to produce the data.
+      // 5.2. Finally, hand off to the stripe reader to produce the data.
       //      This is a sync call that will feed data to the consumer.
       try {
         // TODO: readEncodedColumns is not supposed to throw; errors should be propagated thru
@@ -423,7 +406,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
         // Also, currently readEncodedColumns is not stoppable. The consumer will discard the
         // data it receives for one stripe. We could probably interrupt it, if it checked that.
         stripeReader.readEncodedColumns(stripeIx, stripe, stripeMetadata.getRowIndexes(),
-            stripeMetadata.getEncodings(), stripeMetadata.getStreams(), stripeIncludes,
+            stripeMetadata.getEncodings(), stripeMetadata.getStreams(), globalIncludes,
             colRgs, consumer);
       } catch (Throwable t) {
         consumer.setError(t);
@@ -521,22 +504,14 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
     return HdfsUtils.getFileId(fs, split.getPath(), allowSynthetic);
   }
 
-  private boolean[][] genStripeColRgs(List<Integer> stripeCols, boolean[][] globalColRgs) {
-    boolean[][] stripeColRgs = new boolean[stripeCols.size()][];
-    for (int i = 0, i2 = -1; i < globalColRgs.length; ++i) {
-      if (globalColRgs[i] == null) continue;
-      stripeColRgs[i2] = globalColRgs[i];
-      ++i2;
-    }
-    return stripeColRgs;
-  }
-
   /**
    * Puts all column indexes from metadata to make a column list to read all column.
    */
-  private static List<Integer> createColumnIds(OrcFileMetadata metadata) {
-    List<Integer> columnIds = new ArrayList<Integer>(metadata.getTypes().size());
-    for (int i = 1; i < metadata.getTypes().size(); ++i) {
+  private static List<Integer> getAllColumnIds(OrcFileMetadata metadata) {
+    int rootColumn = OrcInputFormat.getRootColumn(true);
+    List<Integer> types = metadata.getTypes().get(rootColumn).getSubtypesList();
+    List<Integer> columnIds = new ArrayList<Integer>(types.size());
+    for (int i = 0; i < types.size(); ++i) {
       columnIds.add(i);
     }
     return columnIds;
@@ -690,8 +665,9 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
 
   @Override
   public void returnData(OrcEncodedColumnBatch ecb) {
-    for (ColumnStreamData[] datas : ecb.getColumnData()) {
-      if (datas == null) continue;
+    for (int colIx = 0; colIx < ecb.getTotalColCount(); ++colIx) {
+      if (!ecb.hasData(colIx)) continue;
+      ColumnStreamData[] datas = ecb.getColumnData(colIx);
       for (ColumnStreamData data : datas) {
         if (data == null || data.decRef() != 0) continue;
         if (LlapIoImpl.LOCKING_LOGGER.isTraceEnabled()) {
@@ -749,12 +725,17 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
         }
       }
       assert isAll || isNone || rgsToRead.length == rgCount;
-      readState[stripeIxMod] = new boolean[columnIds.size()][];
-      for (int j = 0; j < columnIds.size(); ++j) {
-        readState[stripeIxMod][j] = (isAll || isNone) ? rgsToRead :
+      int fileIncludesCount = 0;
+      // TODO: hacky for now - skip the root 0-s column.
+      //        We don't need separate readState w/o HL cache, should get rid of that instead.
+      for (int includeIx = 1; includeIx < globalIncludes.length; ++includeIx) {
+        fileIncludesCount += (globalIncludes[includeIx] ? 1 : 0);
+      }
+      readState[stripeIxMod] = new boolean[fileIncludesCount][];
+      for (int includeIx = 0; includeIx < fileIncludesCount; ++includeIx) {
+        readState[stripeIxMod][includeIx] = (isAll || isNone) ? rgsToRead :
           Arrays.copyOf(rgsToRead, rgsToRead.length);
       }
-
       adjustRgMetric(rgCount, rgsToRead, isNone, isAll);
     }
     return hasAnyData;

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java
index c9b0a4d..70cba05 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java
@@ -32,7 +32,9 @@ import org.apache.hadoop.hive.ql.io.orc.Reader;
 import org.apache.orc.CompressionKind;
 import org.apache.orc.FileMetadata;
 import org.apache.orc.OrcProto;
+import org.apache.orc.OrcUtils;
 import org.apache.orc.StripeInformation;
+import org.apache.orc.TypeDescription;
 import org.apache.orc.impl.ReaderImpl;
 
 /** ORC file metadata. Currently contains some duplicate info due to how different parts
@@ -222,4 +224,8 @@ public final class OrcFileMetadata extends LlapCacheableBuffer implements FileMe
   public List<OrcProto.ColumnStatistics> getFileStats() {
     return fileStats;
   }
+
+  public TypeDescription getSchema() {
+    return OrcUtils.convertTypeFromProtobuf(this.types, 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java b/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
index c347181..5d5f991 100644
--- a/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
+++ b/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
@@ -292,7 +292,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
      // Pass-thru.
       convertTreeReader.seek(index);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java b/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
index 245e3ce..e46a0a4 100644
--- a/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
+++ b/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
@@ -80,7 +80,7 @@ public class TreeReaderFactory {
       }
     }
 
-    static IntegerReader createIntegerReader(OrcProto.ColumnEncoding.Kind kind,
+    protected static IntegerReader createIntegerReader(OrcProto.ColumnEncoding.Kind kind,
         InStream in,
         boolean signed, boolean skipCorrupt) throws IOException {
       switch (kind) {
@@ -115,7 +115,7 @@ public class TreeReaderFactory {
      * @param index the indexes loaded from the file
      * @throws IOException
      */
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -201,6 +201,10 @@ public class TreeReaderFactory {
     public BitFieldReader getPresent() {
       return present;
     }
+
+    public int getColumnId() {
+      return columnId;
+    }
   }
 
   public static class NullTreeReader extends TreeReader {
@@ -262,7 +266,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -313,7 +317,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -380,7 +384,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -447,7 +451,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -515,7 +519,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -569,7 +573,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -663,7 +667,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -774,7 +778,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -898,7 +902,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -1007,7 +1011,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -1083,7 +1087,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -1197,7 +1201,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       reader.seek(index);
     }
 
@@ -1346,7 +1350,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -1495,7 +1499,7 @@ public class TreeReaderFactory {
     }
 
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       seek(index[columnId]);
     }
 
@@ -1701,7 +1705,7 @@ public class TreeReaderFactory {
     }
   }
 
-  protected static class StructTreeReader extends TreeReader {
+  public static class StructTreeReader extends TreeReader {
     protected final TreeReader[] fields;
 
     protected StructTreeReader(int columnId,
@@ -1719,8 +1723,21 @@ public class TreeReaderFactory {
       }
     }
 
+    public TreeReader[] getChildReaders() {
+      return fields;
+    }
+
+    protected StructTreeReader(int columnId, InStream present,
+        OrcProto.ColumnEncoding encoding, TreeReader[] childReaders) throws IOException {
+      super(columnId, present);
+      if (encoding != null) {
+        checkEncoding(encoding);
+      }
+      this.fields = childReaders;
+    }
+
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       super.seek(index);
       for (TreeReader kid : fields) {
         if (kid != null) {
@@ -1804,8 +1821,17 @@ public class TreeReaderFactory {
       }
     }
 
+    protected UnionTreeReader(int columnId, InStream present,
+        OrcProto.ColumnEncoding encoding, TreeReader[] childReaders) throws IOException {
+      super(columnId, present);
+      if (encoding != null) {
+        checkEncoding(encoding);
+      }
+      this.fields = childReaders;
+    }
+
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       super.seek(index);
       tags.seek(index[columnId]);
       for (TreeReader kid : fields) {
@@ -1877,8 +1903,18 @@ public class TreeReaderFactory {
           skipCorrupt);
     }
 
+    protected ListTreeReader(int columnId, InStream present, InStream data,
+        OrcProto.ColumnEncoding encoding, TreeReader elementReader) throws IOException {
+      super(columnId, present);
+      if (data != null && encoding != null) {
+        checkEncoding(encoding);
+        this.lengths = createIntegerReader(encoding.getKind(), data, false, false);
+      }
+      this.elementReader = elementReader;
+    }
+
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       super.seek(index);
       lengths.seek(index[columnId]);
       elementReader.seek(index);
@@ -1958,8 +1994,20 @@ public class TreeReaderFactory {
       valueReader = createTreeReader(valueType, evolution, included, skipCorrupt);
     }
 
+    protected MapTreeReader(int columnId, InStream present, InStream data,
+        OrcProto.ColumnEncoding encoding, TreeReader keyReader, TreeReader valueReader)
+        throws IOException {
+      super(columnId, present);
+      if (data != null && encoding != null) {
+        checkEncoding(encoding);
+        this.lengths = createIntegerReader(encoding.getKind(), data, false, false);
+      }
+      this.keyReader = keyReader;
+      this.valueReader = valueReader;
+    }
+
     @Override
-    void seek(PositionProvider[] index) throws IOException {
+    public void seek(PositionProvider[] index) throws IOException {
       super.seek(index);
       lengths.seek(index[columnId]);
       keyReader.seek(index);

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index a9dbc3e..2b1d1ce 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -3636,7 +3636,7 @@ public final class Utilities {
    * @param conf - configuration
    * @return false for types not supported by vectorization, true otherwise
    */
-  public static boolean checkLlapIOSupportedTypes(final Configuration conf) {
+  public static boolean checkVectorizerSupportedTypes(final Configuration conf) {
     final String[] readColumnNames = ColumnProjectionUtils.getReadColumnNames(conf);
     final String columnNames = conf.get(serdeConstants.LIST_COLUMNS);
     final String columnTypes = conf.get(serdeConstants.LIST_COLUMN_TYPES);
@@ -3649,7 +3649,7 @@ public final class Utilities {
     final List<String> allColumnNames = Lists.newArrayList(columnNames.split(","));
     final List<TypeInfo> typeInfos = TypeInfoUtils.getTypeInfosFromTypeString(columnTypes);
     final List<String> allColumnTypes = TypeInfoUtils.getTypeStringsFromTypeInfo(typeInfos);
-    return checkLlapIOSupportedTypes(Lists.newArrayList(readColumnNames), allColumnNames,
+    return checkVectorizerSupportedTypes(Lists.newArrayList(readColumnNames), allColumnNames,
         allColumnTypes);
   }
 
@@ -3660,7 +3660,7 @@ public final class Utilities {
    * @param allColumnTypes - all column types
    * @return false for types not supported by vectorization, true otherwise
    */
-  public static boolean checkLlapIOSupportedTypes(final List<String> readColumnNames,
+  public static boolean checkVectorizerSupportedTypes(final List<String> readColumnNames,
       final List<String> allColumnNames, final List<String> allColumnTypes) {
     final String[] readColumnTypes = getReadColumnTypes(readColumnNames, allColumnNames,
         allColumnTypes);

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/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 ccb39da..572953a 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
@@ -282,7 +282,7 @@ public class OrcInputFormat implements InputFormat<NullWritable, OrcStruct>,
    * @param isOriginal is the file in the original format?
    * @return the column number for the root of row.
    */
-  static int getRootColumn(boolean isOriginal) {
+  public static int getRootColumn(boolean isOriginal) {
     return isOriginal ? 0 : (OrcRecordUpdater.ROW + 1);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java
index 4d09dcd..ea9904a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java
@@ -55,4 +55,4 @@ public interface EncodedReader {
    * to just checking the constant in the first place.
    */
   void setTracing(boolean isEnabled);
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/6efa869f/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
index a4925b9..8f0c237 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.io.orc.encoded;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import org.slf4j.Logger;
@@ -42,6 +43,7 @@ import org.apache.orc.impl.RecordReaderUtils;
 import org.apache.orc.impl.StreamName;
 import org.apache.orc.StripeInformation;
 import org.apache.orc.impl.BufferChunk;
+import org.apache.hadoop.hive.llap.DebugUtils;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Reader.OrcEncodedColumnBatch;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Reader.PoolFactory;
 import org.apache.orc.OrcProto;
@@ -127,11 +129,13 @@ class EncodedReaderImpl implements EncodedReader {
 
   /** Helper context for each column being read */
   private static final class ColumnReadContext {
+
     public ColumnReadContext(int colIx, OrcProto.ColumnEncoding encoding,
-                             OrcProto.RowIndex rowIndex) {
+                             OrcProto.RowIndex rowIndex, int colRgIx) {
       this.encoding = encoding;
       this.rowIndex = rowIndex;
       this.colIx = colIx;
+      this.includedIx = colRgIx;
       streamCount = 0;
     }
 
@@ -145,6 +149,8 @@ class EncodedReaderImpl implements EncodedReader {
     OrcProto.RowIndex rowIndex;
     /** Column index in the file. */
     int colIx;
+    /** Column index in the included columns only (for RG masks). */
+    int includedIx;
 
     public void addStream(long offset, OrcProto.Stream stream, int indexIx) {
       streams[streamCount++] = new StreamContext(stream, offset, indexIx);
@@ -154,6 +160,7 @@ class EncodedReaderImpl implements EncodedReader {
     public String toString() {
       StringBuilder sb = new StringBuilder();
       sb.append(" column_index: ").append(colIx);
+      sb.append(" included_index: ").append(includedIx);
       sb.append(" encoding: ").append(encoding);
       sb.append(" stream_count: ").append(streamCount);
       int i = 0;
@@ -200,6 +207,7 @@ class EncodedReaderImpl implements EncodedReader {
       OrcProto.RowIndex[] indexes, List<OrcProto.ColumnEncoding> encodings, List<OrcProto.Stream> streamList,
       boolean[] included, boolean[][] colRgs,
       Consumer<OrcEncodedColumnBatch> consumer) throws IOException {
+    isTracingEnabled = true;
     // Note: for now we don't have to setError here, caller will setError if we throw.
     // We are also not supposed to call setDone, since we are only part of the operation.
     long stripeOffset = stripe.getOffset();
@@ -208,18 +216,26 @@ class EncodedReaderImpl implements EncodedReader {
     // 1.1. Figure out which columns have a present stream
     boolean[] hasNull = RecordReaderUtils.findPresentStreamsByColumn(streamList, types);
     if (isTracingEnabled) {
-      LOG.trace("The following columns have PRESENT streams: " + arrayToString(hasNull));
+      LOG.error("The following columns have PRESENT streams: " + arrayToString(hasNull));
     }
 
     // We assume stream list is sorted by column and that non-data
     // streams do not interleave data streams for the same column.
     // 1.2. With that in mind, determine disk ranges to read/get from cache (not by stream).
-    int colRgIx = -1, lastColIx = -1;
-    ColumnReadContext[] colCtxs = new ColumnReadContext[colRgs.length];
-    boolean[] includedRgs = null;
+    ColumnReadContext[] colCtxs = new ColumnReadContext[included.length];
+    int colRgIx = -1;
+    // Don't create context for the 0-s column.
+    for (int i = 1; i < included.length; ++i) {
+      if (!included[i]) continue;
+      colCtxs[i] = new ColumnReadContext(i, encodings.get(i), indexes[i], ++colRgIx);
+      if (isTracingEnabled) {
+        LOG.error("Creating context: " + colCtxs[i].toString());
+      }
+    }
     boolean isCompressed = (codec != null);
     CreateHelper listToRead = new CreateHelper();
     boolean hasIndexOnlyCols = false;
+    boolean[] includedRgs = null; // Will always be the same for all cols at the moment.
     for (OrcProto.Stream stream : streamList) {
       long length = stream.getLength();
       int colIx = stream.getColumn();
@@ -227,28 +243,17 @@ class EncodedReaderImpl implements EncodedReader {
       if (!included[colIx] || StreamName.getArea(streamKind) != StreamName.Area.DATA) {
         // We have a stream for included column, but in future it might have no data streams.
         // It's more like "has at least one column included that has an index stream".
-        hasIndexOnlyCols = hasIndexOnlyCols | included[colIx];
+        hasIndexOnlyCols = hasIndexOnlyCols || included[colIx];
         if (isTracingEnabled) {
-          LOG.trace("Skipping stream: " + streamKind + " at " + offset + ", " + length);
+          LOG.trace("Skipping stream for column " + colIx + ": "
+              + streamKind + " at " + offset + ", " + length);
         }
         offset += length;
         continue;
       }
-      ColumnReadContext ctx = null;
-      if (lastColIx != colIx) {
-        ++colRgIx;
-        assert colCtxs[colRgIx] == null;
-        lastColIx = colIx;
-        includedRgs = colRgs[colRgIx];
-        ctx = colCtxs[colRgIx] = new ColumnReadContext(
-            colIx, encodings.get(colIx), indexes[colIx]);
-        if (isTracingEnabled) {
-          LOG.trace("Creating context " + colRgIx + " for column " + colIx + ":" + ctx.toString());
-        }
-      } else {
-        ctx = colCtxs[colRgIx];
-        assert ctx != null;
-      }
+      ColumnReadContext ctx = colCtxs[colIx];
+      assert ctx != null;
+      includedRgs = colRgs[ctx.includedIx];
       int indexIx = RecordReaderUtils.getIndexPosition(ctx.encoding.getKind(),
           types.get(colIx).getKind(), streamKind, isCompressed, hasNull[colIx]);
       ctx.addStream(offset, stream, indexIx);
@@ -275,7 +280,7 @@ class EncodedReaderImpl implements EncodedReader {
       // TODO: there may be a bug here. Could there be partial RG filtering on index-only column?
       if (hasIndexOnlyCols && (includedRgs == null)) {
         OrcEncodedColumnBatch ecb = POOLS.ecbPool.take();
-        ecb.init(fileKey, stripeIx, OrcEncodedColumnBatch.ALL_RGS, colRgs.length);
+        ecb.init(fileKey, stripeIx, OrcEncodedColumnBatch.ALL_RGS, included.length);
         consumer.consumeData(ecb);
       } else {
         LOG.warn("Nothing to read for stripe [" + stripe + "]");
@@ -309,8 +314,9 @@ class EncodedReaderImpl implements EncodedReader {
     // 3. For uncompressed case, we need some special processing before read.
     DiskRangeList iter = toRead.next;  // Keep "toRead" list for future use, don't extract().
     if (codec == null) {
-      for (int colIxMod = 0; colIxMod < colRgs.length; ++colIxMod) {
-        ColumnReadContext ctx = colCtxs[colIxMod];
+      for (int colIx = 0; colIx < colCtxs.length; ++colIx) {
+        ColumnReadContext ctx = colCtxs[colIx];
+        if (ctx == null) continue; // This column is not included.
         for (int streamIx = 0; streamIx < ctx.streamCount; ++streamIx) {
           StreamContext sctx = ctx.streams[streamIx];
           DiskRangeList newIter = preReadUncompressedStream(
@@ -334,19 +340,27 @@ class EncodedReaderImpl implements EncodedReader {
       boolean isLastRg = rgIx == rgCount - 1;
       // Create the batch we will use to return data for this RG.
       OrcEncodedColumnBatch ecb = POOLS.ecbPool.take();
-      ecb.init(fileKey, stripeIx, rgIx, colRgs.length);
+      ecb.init(fileKey, stripeIx, rgIx, included.length);
       boolean isRGSelected = true;
-      for (int colIxMod = 0; colIxMod < colRgs.length; ++colIxMod) {
-        // TODO: simplify this now that high-level cache has been removed.
-        if (colRgs[colIxMod] != null && !colRgs[colIxMod][rgIx]) {
+      for (int colIx = 0; colIx < colCtxs.length; ++colIx) {
+        ColumnReadContext ctx = colCtxs[colIx];
+        if (ctx == null) continue; // This column is not included.
+        if (isTracingEnabled) {
+          LOG.trace("ctx: {} rgIx: {} isLastRg: {} rgCount: {}", ctx, rgIx, isLastRg, rgCount);
+        }
+        // TODO: simplify this now that high-level cache has been removed. Same RGs for all cols.
+        if (colRgs[ctx.includedIx] != null && !colRgs[ctx.includedIx][rgIx]) {
           // RG x col filtered.
           isRGSelected = false;
-          continue;
+          if (isTracingEnabled) {
+            LOG.trace("colIxMod: {} rgIx: {} colRgs[{}]: {} colRgs[{}][{}]: {}", ctx.includedIx, rgIx, ctx.includedIx,
+              Arrays.toString(colRgs[ctx.includedIx]), ctx.includedIx, rgIx, colRgs[ctx.includedIx][rgIx]);
+          }
+           continue;
         }
-        ColumnReadContext ctx = colCtxs[colIxMod];
         OrcProto.RowIndexEntry index = ctx.rowIndex.getEntry(rgIx),
             nextIndex = isLastRg ? null : ctx.rowIndex.getEntry(rgIx + 1);
-        ecb.initColumn(colIxMod, ctx.colIx, OrcEncodedColumnBatch.MAX_DATA_STREAMS);
+        ecb.initOrcColumn(ctx.colIx);
         for (int streamIx = 0; streamIx < ctx.streamCount; ++streamIx) {
           StreamContext sctx = ctx.streams[streamIx];
           ColumnStreamData cb = null;
@@ -402,7 +416,7 @@ class EncodedReaderImpl implements EncodedReader {
                 sctx.bufferIter = iter = lastCached;
               }
             }
-            ecb.setStreamData(colIxMod, sctx.kind.getNumber(), cb);
+            ecb.setStreamData(ctx.colIx, sctx.kind.getNumber(), cb);
           } catch (Exception ex) {
             DiskRangeList drl = toRead == null ? null : toRead.next;
             LOG.error("Error getting stream [" + sctx.kind + ", " + ctx.encoding + "] for"


[04/35] hive git commit: HIVE-15220 : WebHCat test driver not capturing end time of test accurately (Deepesh Khandelwal via Thejas Nair)

Posted by se...@apache.org.
HIVE-15220 : WebHCat test driver not capturing end time of test accurately (Deepesh Khandelwal via Thejas Nair)


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

Branch: refs/heads/hive-14535
Commit: 085650e0cb5e9c4be219673c4909db3a8c4a979d
Parents: 5aff550
Author: Deepesh Khandelwal <de...@hotmail.com>
Authored: Wed Nov 16 10:43:32 2016 -0800
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Wed Nov 16 10:43:32 2016 -0800

----------------------------------------------------------------------
 hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/085650e0/hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm
----------------------------------------------------------------------
diff --git a/hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm b/hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm
index 2eac653..b965eec 100644
--- a/hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm
+++ b/hcatalog/src/test/e2e/templeton/drivers/TestDriverCurl.pm
@@ -1415,7 +1415,6 @@ sub run
         my ($testResult, $benchmarkResult);
         eval {
           $testResult = $self->runTest(\%testHash, $log);
-          $endTime = time;
           $benchmarkResult = $self->generateBenchmark(\%testHash, $log);
           my $result =
             $self->compare($testResult, $benchmarkResult, $log, \%testHash);
@@ -1433,6 +1432,7 @@ sub run
             $testStatuses->{$testName} = $failedStr;
 
           }
+          $endTime = time;
           $msg .= "\nEnding test $testName at " . $endTime ."\n";
           #print $msg;
           print $log $msg;


[08/35] hive git commit: HIVE-10901: Optimize multi column distinct queries (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-10901: Optimize multi column distinct queries (Pengcheng Xiong, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hive-14535
Commit: 67c022f05f5405abd996446fefc90e7ae34c960d
Parents: 0025b9d
Author: Pengcheng Xiong <px...@apache.org>
Authored: Thu Nov 17 10:48:06 2016 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Thu Nov 17 10:48:06 2016 -0800

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../rules/HiveAggregateProjectMergeRule.java    |  14 ++
 .../rules/HiveExpandDistinctAggregatesRule.java | 206 ++++++++++++++++++-
 .../clientpositive/multi_count_distinct.q       |  38 ++++
 .../llap/vector_grouping_sets.q.out             |   8 +-
 .../results/clientpositive/perf/query70.q.out   |  52 ++---
 .../tez/multi_count_distinct.q.out              | 193 +++++++++++++++++
 .../clientpositive/vector_grouping_sets.q.out   |   8 +-
 8 files changed, 483 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/67c022f0/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index f30152b..e4910e4 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -51,6 +51,7 @@ minitez.query.files=explainuser_3.q,\
   explainanalyze_5.q,\
   hybridgrace_hashjoin_1.q,\
   hybridgrace_hashjoin_2.q,\
+  multi_count_distinct.q,\
   tez_union_with_udf.q
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/67c022f0/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
index c243266..7c3b4b0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
@@ -31,6 +31,7 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 
 import com.google.common.collect.ImmutableList;
@@ -59,6 +60,19 @@ public class HiveAggregateProjectMergeRule extends RelOptRule {
   }
 
   @Override
+  public boolean matches(RelOptRuleCall call) {
+    final Aggregate aggregate = call.rel(0);
+    // Rule cannot be applied if there are GroupingId because it will change the
+    // value as the position will be changed.
+    for (AggregateCall aggCall : aggregate.getAggCallList()) {
+      if (aggCall.getAggregation().equals(HiveGroupingID.INSTANCE)) {
+        return false;
+      }
+    }
+    return super.matches(call);
+  }
+
+  @Override
   public void onMatch(RelOptRuleCall call) {
     final HiveAggregate aggregate = call.rel(0);
     final HiveProject project = call.rel(1);

http://git-wip-us.apache.org/repos/asf/hive/blob/67c022f0/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExpandDistinctAggregatesRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExpandDistinctAggregatesRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExpandDistinctAggregatesRule.java
index 7d4411a..417eda3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExpandDistinctAggregatesRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveExpandDistinctAggregatesRule.java
@@ -16,13 +16,17 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
+import java.math.BigDecimal;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
@@ -31,16 +35,30 @@ import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.metadata.RelColumnOrigin;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
+import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveRelNode;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -73,6 +91,8 @@ public final class HiveExpandDistinctAggregatesRule extends RelOptRule {
           HiveRelFactories.HIVE_PROJECT_FACTORY);
 
   private static RelFactories.ProjectFactory projFactory;
+  
+  protected static final Logger LOG = LoggerFactory.getLogger(HiveExpandDistinctAggregatesRule.class);
 
   //~ Constructors -----------------------------------------------------------
 
@@ -82,19 +102,25 @@ public final class HiveExpandDistinctAggregatesRule extends RelOptRule {
     projFactory = projectFactory;
   }
 
+  RelOptCluster cluster = null;
+  RexBuilder rexBuilder = null;
+
   //~ Methods ----------------------------------------------------------------
 
   @Override
   public void onMatch(RelOptRuleCall call) {
     final Aggregate aggregate = call.rel(0);
-    if (!aggregate.containsDistinctCall()) {
+    int numCountDistinct = getNumCountDistinctCall(aggregate);
+    if (numCountDistinct == 0) {
       return;
     }
 
-    // Find all of the agg expressions. We use a LinkedHashSet to ensure
-    // determinism.
+    // Find all of the agg expressions. We use a List (for all count(distinct))
+    // as well as a Set (for all others) to ensure determinism.
     int nonDistinctCount = 0;
+    List<List<Integer>> argListList = new ArrayList<List<Integer>>();
     Set<List<Integer>> argListSets = new LinkedHashSet<List<Integer>>();
+    Set<Integer> positions = new HashSet<>();
     for (AggregateCall aggCall : aggregate.getAggCallList()) {
       if (!aggCall.isDistinct()) {
         ++nonDistinctCount;
@@ -103,11 +129,35 @@ public final class HiveExpandDistinctAggregatesRule extends RelOptRule {
       ArrayList<Integer> argList = new ArrayList<Integer>();
       for (Integer arg : aggCall.getArgList()) {
         argList.add(arg);
+        positions.add(arg);
       }
+      // Aggr checks for sorted argList.
+      argListList.add(argList);
       argListSets.add(argList);
     }
     Util.permAssert(argListSets.size() > 0, "containsDistinctCall lied");
 
+    if (numCountDistinct > 1 && numCountDistinct == aggregate.getAggCallList().size()
+        && aggregate.getGroupSet().isEmpty()) {
+      // now positions contains all the distinct positions, i.e., $5, $4, $6
+      // we need to first sort them as group by set
+      // and then get their position later, i.e., $4->1, $5->2, $6->3
+      cluster = aggregate.getCluster();
+      rexBuilder = cluster.getRexBuilder();
+      RelNode converted = null;
+      List<Integer> sourceOfForCountDistinct = new ArrayList<>();
+      sourceOfForCountDistinct.addAll(positions);
+      Collections.sort(sourceOfForCountDistinct);
+      try {
+        converted = convert(aggregate, argListList, sourceOfForCountDistinct);
+      } catch (CalciteSemanticException e) {
+        LOG.debug(e.toString());
+        throw new RuntimeException(e);
+      }
+      call.transformTo(converted);
+      return;
+    }
+
     // If all of the agg expressions are distinct and have the same
     // arguments then we can use a more efficient form.
     if ((nonDistinctCount == 0) && (argListSets.size() == 1)) {
@@ -133,6 +183,156 @@ public final class HiveExpandDistinctAggregatesRule extends RelOptRule {
   }
 
   /**
+   * Converts an aggregate relational expression that contains only
+   * count(distinct) to grouping sets with count. For example select
+   * count(distinct department_id), count(distinct gender), count(distinct
+   * education_level) from employee; can be transformed to select count(case i
+   * when 1 then 1 else null end) as c0, count(case i when 2 then 1 else null
+   * end) as c1, count(case i when 4 then 1 else null end) as c2 from (select
+   * grouping__id as i, department_id, gender, education_level from employee
+   * group by department_id, gender, education_level grouping sets
+   * (department_id, gender, education_level))subq;
+   * @throws CalciteSemanticException 
+   */
+  private RelNode convert(Aggregate aggregate, List<List<Integer>> argList, List<Integer> sourceOfForCountDistinct) throws CalciteSemanticException {
+    // we use this map to map the position of argList to the position of grouping set
+    Map<Integer, Integer> map = new HashMap<>();
+    List<List<Integer>> cleanArgList = new ArrayList<>();
+    final Aggregate groupingSets = createGroupingSets(aggregate, argList, cleanArgList, map, sourceOfForCountDistinct);
+    return createCount(groupingSets, argList, cleanArgList, map, sourceOfForCountDistinct);
+  }
+
+  private long getGroupingIdValue(List<Integer> list, List<Integer> sourceOfForCountDistinct) {
+    long ind = 0;
+    for (int i : list) {
+      ind |= 1 << sourceOfForCountDistinct.indexOf(i);
+    }
+    return ind;
+  }
+
+  /**
+   * @param aggr: the original aggregate
+   * @param argList: the original argList in aggregate
+   * @param cleanArgList: the new argList without duplicates
+   * @param map: the mapping from the original argList to the new argList
+   * @param sourceOfForCountDistinct: the sorted positions of groupset
+   * @return
+   * @throws CalciteSemanticException
+   */
+  private RelNode createCount(Aggregate aggr, List<List<Integer>> argList,
+      List<List<Integer>> cleanArgList, Map<Integer, Integer> map,
+      List<Integer> sourceOfForCountDistinct) throws CalciteSemanticException {
+    List<RexNode> originalInputRefs = Lists.transform(aggr.getRowType().getFieldList(),
+        new Function<RelDataTypeField, RexNode>() {
+          @Override
+          public RexNode apply(RelDataTypeField input) {
+            return new RexInputRef(input.getIndex(), input.getType());
+          }
+        });
+    final List<RexNode> gbChildProjLst = Lists.newArrayList();
+    for (List<Integer> list : cleanArgList) {
+      RexNode equal = rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+          originalInputRefs.get(originalInputRefs.size() - 1),
+          rexBuilder.makeExactLiteral(new BigDecimal(getGroupingIdValue(list, sourceOfForCountDistinct))));
+      RexNode condition = rexBuilder.makeCall(SqlStdOperatorTable.CASE, equal,
+          rexBuilder.makeExactLiteral(BigDecimal.ONE), rexBuilder.constantNull());
+      gbChildProjLst.add(condition);
+    }
+
+    // create the project before GB
+    RelNode gbInputRel = HiveProject.create(aggr, gbChildProjLst, null);
+
+    // create the aggregate
+    List<AggregateCall> aggregateCalls = Lists.newArrayList();
+    RelDataType aggFnRetType = TypeConverter.convert(TypeInfoFactory.longTypeInfo,
+        cluster.getTypeFactory());
+    for (int i = 0; i < cleanArgList.size(); i++) {
+      AggregateCall aggregateCall = HiveCalciteUtil.createSingleArgAggCall("count", cluster,
+          TypeInfoFactory.longTypeInfo, i, aggFnRetType);
+      aggregateCalls.add(aggregateCall);
+    }
+    Aggregate aggregate = new HiveAggregate(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION), gbInputRel,
+        false, ImmutableBitSet.of(), null, aggregateCalls);
+
+    // create the project after GB. For those repeated values, e.g., select
+    // count(distinct x, y), count(distinct y, x), we find the correct mapping.
+    if (map.isEmpty()) {
+      return aggregate;
+    } else {
+      List<RexNode> originalAggrRefs = Lists.transform(aggregate.getRowType().getFieldList(),
+          new Function<RelDataTypeField, RexNode>() {
+            @Override
+            public RexNode apply(RelDataTypeField input) {
+              return new RexInputRef(input.getIndex(), input.getType());
+            }
+          });
+      final List<RexNode> projLst = Lists.newArrayList();
+      int index = 0;
+      for (int i = 0; i < argList.size(); i++) {
+        if (map.containsKey(i)) {
+          projLst.add(originalAggrRefs.get(map.get(i)));
+        } else {
+          projLst.add(originalAggrRefs.get(index++));
+        }
+      }
+      return HiveProject.create(aggregate, projLst, null);
+    }
+  }
+
+  /**
+   * @param aggregate: the original aggregate
+   * @param argList: the original argList in aggregate
+   * @param cleanArgList: the new argList without duplicates
+   * @param map: the mapping from the original argList to the new argList
+   * @param sourceOfForCountDistinct: the sorted positions of groupset
+   * @return
+   */
+  private Aggregate createGroupingSets(Aggregate aggregate, List<List<Integer>> argList,
+      List<List<Integer>> cleanArgList, Map<Integer, Integer> map,
+      List<Integer> sourceOfForCountDistinct) {
+    final ImmutableBitSet groupSet = ImmutableBitSet.of(sourceOfForCountDistinct);
+    final List<ImmutableBitSet> origGroupSets = new ArrayList<>();
+
+    for (int i = 0; i < argList.size(); i++) {
+      List<Integer> list = argList.get(i);
+      ImmutableBitSet bitSet = ImmutableBitSet.of(list);
+      int prev = origGroupSets.indexOf(bitSet);
+      if (prev == -1) {
+        origGroupSets.add(bitSet);
+        cleanArgList.add(list);
+      } else {
+        map.put(i, prev);
+      }
+    }
+    // Calcite expects the grouping sets sorted and without duplicates
+    Collections.sort(origGroupSets, ImmutableBitSet.COMPARATOR);
+
+    List<AggregateCall> aggregateCalls = new ArrayList<AggregateCall>();
+    // Create GroupingID column
+    AggregateCall aggCall = AggregateCall.create(HiveGroupingID.INSTANCE, false,
+        new ImmutableList.Builder<Integer>().build(), -1, this.cluster.getTypeFactory()
+            .createSqlType(SqlTypeName.INTEGER), HiveGroupingID.INSTANCE.getName());
+    aggregateCalls.add(aggCall);
+    return new HiveAggregate(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+        aggregate.getInput(), true, groupSet, origGroupSets, aggregateCalls);
+  }
+
+  /**
+   * Returns the number of count DISTINCT
+   *
+   * @return the number of count DISTINCT
+   */
+  private int getNumCountDistinctCall(Aggregate hiveAggregate) {
+    int cnt = 0;
+    for (AggregateCall aggCall : hiveAggregate.getAggCallList()) {
+      if (aggCall.isDistinct() && (aggCall.getAggregation().getName().equalsIgnoreCase("count"))) {
+        cnt++;
+      }
+    }
+    return cnt;
+  }
+
+  /**
    * Converts an aggregate relational expression that contains just one
    * distinct aggregate function (or perhaps several over the same arguments)
    * and no non-distinct aggregate functions.

http://git-wip-us.apache.org/repos/asf/hive/blob/67c022f0/ql/src/test/queries/clientpositive/multi_count_distinct.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/multi_count_distinct.q b/ql/src/test/queries/clientpositive/multi_count_distinct.q
new file mode 100644
index 0000000..855cb64
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/multi_count_distinct.q
@@ -0,0 +1,38 @@
+set hive.mapred.mode=nonstrict;
+
+drop table employee;
+
+create table employee (department_id int, gender varchar(10), education_level int);
+ 
+insert into employee values (1, 'M', 1),(1, 'M', 1),(2, 'F', 1),(1, 'F', 3),(1, 'M', 2),(4, 'M', 1),(2, 'F', 1),(2, 'F', 3),(3, 'M', 2);
+
+explain select count(distinct department_id), count(distinct gender), count(distinct education_level) from employee;
+
+select count(distinct department_id), count(distinct gender), count(distinct education_level) from employee;
+
+select count(distinct department_id), count(distinct gender), count(distinct education_level), count(distinct education_level) from employee;
+
+select count(distinct department_id), count(distinct gender), count(distinct education_level), 
+count(distinct education_level, department_id) from employee;
+
+select count(distinct gender), count(distinct department_id), count(distinct gender), count(distinct education_level),
+count(distinct education_level, department_id), count(distinct department_id, education_level) from employee;
+
+explain select count(distinct gender), count(distinct department_id), count(distinct gender), count(distinct education_level),
+count(distinct education_level, department_id), count(distinct department_id, education_level), count(distinct department_id, education_level, gender) from employee;
+
+select count(distinct gender), count(distinct department_id), count(distinct gender), count(distinct education_level),
+count(distinct education_level, department_id), count(distinct department_id, education_level), count(distinct department_id, education_level, gender) from employee;
+
+select count(case i when 1 then 1 else null end) as c0, count(case i when 2 then 1 else null end) as c1, 
+count(case i when 4 then 1 else null end) as c2 from (select grouping__id as i, department_id, gender, 
+education_level from employee group by department_id, gender, education_level grouping sets 
+(department_id, gender, education_level))subq;
+
+select grouping__id as i, department_id, gender, education_level from employee 
+group by department_id, gender, education_level grouping sets 
+(department_id, gender, education_level, (education_level, department_id));
+
+
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/67c022f0/ql/src/test/results/clientpositive/llap/vector_grouping_sets.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_grouping_sets.q.out b/ql/src/test/results/clientpositive/llap/vector_grouping_sets.q.out
index 2cd758c..127478e 100644
--- a/ql/src/test/results/clientpositive/llap/vector_grouping_sets.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_grouping_sets.q.out
@@ -241,10 +241,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: s_store_id (type: string)
-                    outputColumnNames: s_store_id
+                    outputColumnNames: _col0
                     Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: s_store_id (type: string), '0' (type: string)
+                      keys: _col0 (type: string), '0' (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE
@@ -329,10 +329,10 @@ STAGE PLANS:
                   Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: s_store_id (type: string)
-                    outputColumnNames: s_store_id
+                    outputColumnNames: _col0
                     Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      keys: s_store_id (type: string), '0' (type: string)
+                      keys: _col0 (type: string), '0' (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/67c022f0/ql/src/test/results/clientpositive/perf/query70.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query70.q.out b/ql/src/test/results/clientpositive/perf/query70.q.out
index 5a82cd1..611af74 100644
--- a/ql/src/test/results/clientpositive/perf/query70.q.out
+++ b/ql/src/test/results/clientpositive/perf/query70.q.out
@@ -21,46 +21,46 @@ Stage-0
     limit:100
     Stage-1
       Reducer 6
-      File Output Operator [FS_62]
-        Limit [LIM_61] (rows=100 width=88)
+      File Output Operator [FS_63]
+        Limit [LIM_62] (rows=100 width=88)
           Number of rows:100
-          Select Operator [SEL_60] (rows=1045432122 width=88)
+          Select Operator [SEL_61] (rows=1045432122 width=88)
             Output:["_col0","_col1","_col2","_col3","_col4"]
           <-Reducer 5 [SIMPLE_EDGE]
-            SHUFFLE [RS_59]
-              Select Operator [SEL_57] (rows=1045432122 width=88)
+            SHUFFLE [RS_60]
+              Select Operator [SEL_58] (rows=1045432122 width=88)
                 Output:["_col0","_col1","_col2","_col3","_col4"]
-                PTF Operator [PTF_56] (rows=1045432122 width=88)
+                PTF Operator [PTF_57] (rows=1045432122 width=88)
                   Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col4 ASC NULLS FIRST","partition by:":"_col5, CASE WHEN ((_col5 = 2)) THEN (_col0) ELSE (null) END"}]
-                  Select Operator [SEL_55] (rows=1045432122 width=88)
+                  Select Operator [SEL_56] (rows=1045432122 width=88)
                     Output:["_col0","_col1","_col4","_col5"]
                   <-Reducer 4 [SIMPLE_EDGE]
-                    SHUFFLE [RS_54]
+                    SHUFFLE [RS_55]
                       PartitionCols:_col5, CASE WHEN ((_col5 = 2)) THEN (_col0) ELSE (null) END
-                      Select Operator [SEL_53] (rows=1045432122 width=88)
+                      Select Operator [SEL_54] (rows=1045432122 width=88)
                         Output:["_col0","_col1","_col4","_col5"]
-                        Group By Operator [GBY_52] (rows=1045432122 width=88)
+                        Group By Operator [GBY_53] (rows=1045432122 width=88)
                           Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                         <-Reducer 3 [SIMPLE_EDGE]
-                          SHUFFLE [RS_51]
+                          SHUFFLE [RS_52]
                             PartitionCols:_col0, _col1, _col2
-                            Group By Operator [GBY_50] (rows=2090864244 width=88)
-                              Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col2)"],keys:_col6, _col7, '0'
+                            Group By Operator [GBY_51] (rows=2090864244 width=88)
+                              Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col2)"],keys:_col0, _col1, '0'
                               Select Operator [SEL_49] (rows=696954748 width=88)
-                                Output:["_col6","_col7","_col2"]
-                                Merge Join Operator [MERGEJOIN_91] (rows=696954748 width=88)
+                                Output:["_col0","_col1","_col2"]
+                                Merge Join Operator [MERGEJOIN_92] (rows=696954748 width=88)
                                   Conds:RS_46._col1=RS_47._col0(Inner),Output:["_col2","_col6","_col7"]
                                 <-Reducer 2 [SIMPLE_EDGE]
                                   SHUFFLE [RS_46]
                                     PartitionCols:_col1
-                                    Merge Join Operator [MERGEJOIN_87] (rows=633595212 width=88)
+                                    Merge Join Operator [MERGEJOIN_88] (rows=633595212 width=88)
                                       Conds:RS_43._col0=RS_44._col0(Inner),Output:["_col1","_col2"]
                                     <-Map 1 [SIMPLE_EDGE]
                                       SHUFFLE [RS_43]
                                         PartitionCols:_col0
                                         Select Operator [SEL_2] (rows=575995635 width=88)
                                           Output:["_col0","_col1","_col2"]
-                                          Filter Operator [FIL_80] (rows=575995635 width=88)
+                                          Filter Operator [FIL_81] (rows=575995635 width=88)
                                             predicate:(ss_sold_date_sk is not null and ss_store_sk is not null)
                                             TableScan [TS_0] (rows=575995635 width=88)
                                               default@store_sales,ss,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_store_sk","ss_net_profit"]
@@ -69,21 +69,21 @@ Stage-0
                                         PartitionCols:_col0
                                         Select Operator [SEL_5] (rows=8116 width=1119)
                                           Output:["_col0"]
-                                          Filter Operator [FIL_81] (rows=8116 width=1119)
+                                          Filter Operator [FIL_82] (rows=8116 width=1119)
                                             predicate:(d_month_seq BETWEEN 1193 AND 1204 and d_date_sk is not null)
                                             TableScan [TS_3] (rows=73049 width=1119)
                                               default@date_dim,d1,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_month_seq"]
                                 <-Reducer 9 [SIMPLE_EDGE]
                                   SHUFFLE [RS_47]
                                     PartitionCols:_col0
-                                    Merge Join Operator [MERGEJOIN_90] (rows=127775039 width=88)
+                                    Merge Join Operator [MERGEJOIN_91] (rows=127775039 width=88)
                                       Conds:RS_39._col2=RS_40._col0(Left Semi),Output:["_col0","_col1","_col2"]
                                     <-Map 8 [SIMPLE_EDGE]
                                       SHUFFLE [RS_39]
                                         PartitionCols:_col2
                                         Select Operator [SEL_8] (rows=1704 width=1910)
                                           Output:["_col0","_col1","_col2"]
-                                          Filter Operator [FIL_82] (rows=1704 width=1910)
+                                          Filter Operator [FIL_83] (rows=1704 width=1910)
                                             predicate:(s_store_sk is not null and s_state is not null)
                                             TableScan [TS_6] (rows=1704 width=1910)
                                               default@store,s,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_county","s_state"]
@@ -94,7 +94,7 @@ Stage-0
                                           Output:["_col0"],keys:_col0
                                           Select Operator [SEL_32] (rows=116159124 width=88)
                                             Output:["_col0"]
-                                            Filter Operator [FIL_83] (rows=116159124 width=88)
+                                            Filter Operator [FIL_84] (rows=116159124 width=88)
                                               predicate:(rank_window_0 <= 5)
                                               PTF Operator [PTF_31] (rows=348477374 width=88)
                                                 Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 DESC NULLS LAST","partition by:":"_col0"}]
@@ -112,28 +112,28 @@ Stage-0
                                                           Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col6
                                                           Select Operator [SEL_24] (rows=696954748 width=88)
                                                             Output:["_col6","_col2"]
-                                                            Merge Join Operator [MERGEJOIN_89] (rows=696954748 width=88)
+                                                            Merge Join Operator [MERGEJOIN_90] (rows=696954748 width=88)
                                                               Conds:RS_21._col1=RS_22._col0(Inner),Output:["_col2","_col6"]
                                                             <-Map 16 [SIMPLE_EDGE]
                                                               SHUFFLE [RS_22]
                                                                 PartitionCols:_col0
                                                                 Select Operator [SEL_17] (rows=1704 width=1910)
                                                                   Output:["_col0","_col1"]
-                                                                  Filter Operator [FIL_86] (rows=1704 width=1910)
+                                                                  Filter Operator [FIL_87] (rows=1704 width=1910)
                                                                     predicate:(s_store_sk is not null and s_state is not null)
                                                                     TableScan [TS_15] (rows=1704 width=1910)
                                                                       default@store,store,Tbl:COMPLETE,Col:NONE,Output:["s_store_sk","s_state"]
                                                             <-Reducer 11 [SIMPLE_EDGE]
                                                               SHUFFLE [RS_21]
                                                                 PartitionCols:_col1
-                                                                Merge Join Operator [MERGEJOIN_88] (rows=633595212 width=88)
+                                                                Merge Join Operator [MERGEJOIN_89] (rows=633595212 width=88)
                                                                   Conds:RS_18._col0=RS_19._col0(Inner),Output:["_col1","_col2"]
                                                                 <-Map 10 [SIMPLE_EDGE]
                                                                   SHUFFLE [RS_18]
                                                                     PartitionCols:_col0
                                                                     Select Operator [SEL_11] (rows=575995635 width=88)
                                                                       Output:["_col0","_col1","_col2"]
-                                                                      Filter Operator [FIL_84] (rows=575995635 width=88)
+                                                                      Filter Operator [FIL_85] (rows=575995635 width=88)
                                                                         predicate:(ss_store_sk is not null and ss_sold_date_sk is not null)
                                                                         TableScan [TS_9] (rows=575995635 width=88)
                                                                           default@store_sales,store_sales,Tbl:COMPLETE,Col:NONE,Output:["ss_sold_date_sk","ss_store_sk","ss_net_profit"]
@@ -142,7 +142,7 @@ Stage-0
                                                                     PartitionCols:_col0
                                                                     Select Operator [SEL_14] (rows=8116 width=1119)
                                                                       Output:["_col0"]
-                                                                      Filter Operator [FIL_85] (rows=8116 width=1119)
+                                                                      Filter Operator [FIL_86] (rows=8116 width=1119)
                                                                         predicate:(d_month_seq BETWEEN 1193 AND 1204 and d_date_sk is not null)
                                                                         TableScan [TS_12] (rows=73049 width=1119)
                                                                           default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_month_seq"]

http://git-wip-us.apache.org/repos/asf/hive/blob/67c022f0/ql/src/test/results/clientpositive/tez/multi_count_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/multi_count_distinct.q.out b/ql/src/test/results/clientpositive/tez/multi_count_distinct.q.out
new file mode 100644
index 0000000..b623f1d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/multi_count_distinct.q.out
@@ -0,0 +1,193 @@
+PREHOOK: query: drop table employee
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table employee
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table employee (department_id int, gender varchar(10), education_level int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@employee
+POSTHOOK: query: create table employee (department_id int, gender varchar(10), education_level int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@employee
+PREHOOK: query: insert into employee values (1, 'M', 1),(1, 'M', 1),(2, 'F', 1),(1, 'F', 3),(1, 'M', 2),(4, 'M', 1),(2, 'F', 1),(2, 'F', 3),(3, 'M', 2)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@employee
+POSTHOOK: query: insert into employee values (1, 'M', 1),(1, 'M', 1),(2, 'F', 1),(1, 'F', 3),(1, 'M', 2),(4, 'M', 1),(2, 'F', 1),(2, 'F', 3),(3, 'M', 2)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@employee
+POSTHOOK: Lineage: employee.department_id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: employee.education_level EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: employee.gender EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: explain select count(distinct department_id), count(distinct gender), count(distinct education_level) from employee
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select count(distinct department_id), count(distinct gender), count(distinct education_level) from employee
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 3
+      File Output Operator [FS_12]
+        Group By Operator [GBY_10] (rows=1 width=24)
+          Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
+        <-Reducer 2 [SIMPLE_EDGE]
+          SHUFFLE [RS_9]
+            Group By Operator [GBY_8] (rows=1 width=24)
+              Output:["_col0","_col1","_col2"],aggregations:["count(_col0)","count(_col1)","count(_col2)"]
+              Select Operator [SEL_6] (rows=13 width=5)
+                Output:["_col0","_col1","_col2"]
+                Group By Operator [GBY_5] (rows=13 width=5)
+                  Output:["_col0","_col1","_col2","_col3"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
+                <-Map 1 [SIMPLE_EDGE]
+                  SHUFFLE [RS_4]
+                    PartitionCols:_col0, _col1, _col2, _col3
+                    Group By Operator [GBY_3] (rows=27 width=5)
+                      Output:["_col0","_col1","_col2","_col3"],keys:_col0, _col1, _col2, '0'
+                      Select Operator [SEL_1] (rows=9 width=5)
+                        Output:["_col0","_col1","_col2"]
+                        TableScan [TS_0] (rows=9 width=5)
+                          default@employee,employee,Tbl:COMPLETE,Col:NONE,Output:["department_id","gender","education_level"]
+
+PREHOOK: query: select count(distinct department_id), count(distinct gender), count(distinct education_level) from employee
+PREHOOK: type: QUERY
+PREHOOK: Input: default@employee
+#### A masked pattern was here ####
+POSTHOOK: query: select count(distinct department_id), count(distinct gender), count(distinct education_level) from employee
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@employee
+#### A masked pattern was here ####
+4	2	3
+PREHOOK: query: select count(distinct department_id), count(distinct gender), count(distinct education_level), count(distinct education_level) from employee
+PREHOOK: type: QUERY
+PREHOOK: Input: default@employee
+#### A masked pattern was here ####
+POSTHOOK: query: select count(distinct department_id), count(distinct gender), count(distinct education_level), count(distinct education_level) from employee
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@employee
+#### A masked pattern was here ####
+4	2	3	3
+PREHOOK: query: select count(distinct department_id), count(distinct gender), count(distinct education_level), 
+count(distinct education_level, department_id) from employee
+PREHOOK: type: QUERY
+PREHOOK: Input: default@employee
+#### A masked pattern was here ####
+POSTHOOK: query: select count(distinct department_id), count(distinct gender), count(distinct education_level), 
+count(distinct education_level, department_id) from employee
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@employee
+#### A masked pattern was here ####
+4	2	3	7
+PREHOOK: query: select count(distinct gender), count(distinct department_id), count(distinct gender), count(distinct education_level),
+count(distinct education_level, department_id), count(distinct department_id, education_level) from employee
+PREHOOK: type: QUERY
+PREHOOK: Input: default@employee
+#### A masked pattern was here ####
+POSTHOOK: query: select count(distinct gender), count(distinct department_id), count(distinct gender), count(distinct education_level),
+count(distinct education_level, department_id), count(distinct department_id, education_level) from employee
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@employee
+#### A masked pattern was here ####
+2	4	2	3	7	7
+PREHOOK: query: explain select count(distinct gender), count(distinct department_id), count(distinct gender), count(distinct education_level),
+count(distinct education_level, department_id), count(distinct department_id, education_level), count(distinct department_id, education_level, gender) from employee
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select count(distinct gender), count(distinct department_id), count(distinct gender), count(distinct education_level),
+count(distinct education_level, department_id), count(distinct department_id, education_level), count(distinct department_id, education_level, gender) from employee
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 3
+      File Output Operator [FS_12]
+        Select Operator [SEL_11] (rows=1 width=40)
+          Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
+          Group By Operator [GBY_10] (rows=1 width=40)
+            Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)","count(VALUE._col3)","count(VALUE._col4)"]
+          <-Reducer 2 [SIMPLE_EDGE]
+            SHUFFLE [RS_9]
+              Group By Operator [GBY_8] (rows=1 width=40)
+                Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["count(_col0)","count(_col1)","count(_col2)","count(_col3)","count(_col4)"]
+                Select Operator [SEL_6] (rows=22 width=5)
+                  Output:["_col0","_col1","_col2","_col3","_col4"]
+                  Group By Operator [GBY_5] (rows=22 width=5)
+                    Output:["_col0","_col1","_col2","_col3"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3
+                  <-Map 1 [SIMPLE_EDGE]
+                    SHUFFLE [RS_4]
+                      PartitionCols:_col0, _col1, _col2, _col3
+                      Group By Operator [GBY_3] (rows=45 width=5)
+                        Output:["_col0","_col1","_col2","_col3"],keys:_col0, _col1, _col2, '0'
+                        Select Operator [SEL_1] (rows=9 width=5)
+                          Output:["_col0","_col1","_col2"]
+                          TableScan [TS_0] (rows=9 width=5)
+                            default@employee,employee,Tbl:COMPLETE,Col:NONE,Output:["gender","department_id","education_level"]
+
+PREHOOK: query: select count(distinct gender), count(distinct department_id), count(distinct gender), count(distinct education_level),
+count(distinct education_level, department_id), count(distinct department_id, education_level), count(distinct department_id, education_level, gender) from employee
+PREHOOK: type: QUERY
+PREHOOK: Input: default@employee
+#### A masked pattern was here ####
+POSTHOOK: query: select count(distinct gender), count(distinct department_id), count(distinct gender), count(distinct education_level),
+count(distinct education_level, department_id), count(distinct department_id, education_level), count(distinct department_id, education_level, gender) from employee
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@employee
+#### A masked pattern was here ####
+2	4	2	3	7	7	7
+PREHOOK: query: select count(case i when 1 then 1 else null end) as c0, count(case i when 2 then 1 else null end) as c1, 
+count(case i when 4 then 1 else null end) as c2 from (select grouping__id as i, department_id, gender, 
+education_level from employee group by department_id, gender, education_level grouping sets 
+(department_id, gender, education_level))subq
+PREHOOK: type: QUERY
+PREHOOK: Input: default@employee
+#### A masked pattern was here ####
+POSTHOOK: query: select count(case i when 1 then 1 else null end) as c0, count(case i when 2 then 1 else null end) as c1, 
+count(case i when 4 then 1 else null end) as c2 from (select grouping__id as i, department_id, gender, 
+education_level from employee group by department_id, gender, education_level grouping sets 
+(department_id, gender, education_level))subq
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@employee
+#### A masked pattern was here ####
+4	2	3
+PREHOOK: query: select grouping__id as i, department_id, gender, education_level from employee 
+group by department_id, gender, education_level grouping sets 
+(department_id, gender, education_level, (education_level, department_id))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@employee
+#### A masked pattern was here ####
+POSTHOOK: query: select grouping__id as i, department_id, gender, education_level from employee 
+group by department_id, gender, education_level grouping sets 
+(department_id, gender, education_level, (education_level, department_id))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@employee
+#### A masked pattern was here ####
+4	NULL	NULL	1
+4	NULL	NULL	2
+4	NULL	NULL	3
+2	NULL	F	NULL
+2	NULL	M	NULL
+1	1	NULL	NULL
+5	1	NULL	1
+5	1	NULL	2
+5	1	NULL	3
+1	2	NULL	NULL
+5	2	NULL	1
+5	2	NULL	3
+1	3	NULL	NULL
+5	3	NULL	2
+1	4	NULL	NULL
+5	4	NULL	1

http://git-wip-us.apache.org/repos/asf/hive/blob/67c022f0/ql/src/test/results/clientpositive/vector_grouping_sets.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_grouping_sets.q.out b/ql/src/test/results/clientpositive/vector_grouping_sets.q.out
index 9e35049..aa9ee87 100644
--- a/ql/src/test/results/clientpositive/vector_grouping_sets.q.out
+++ b/ql/src/test/results/clientpositive/vector_grouping_sets.q.out
@@ -225,10 +225,10 @@ STAGE PLANS:
             Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: s_store_id (type: string)
-              outputColumnNames: s_store_id
+              outputColumnNames: _col0
               Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: s_store_id (type: string), '0' (type: string)
+                keys: _col0 (type: string), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE
@@ -303,10 +303,10 @@ STAGE PLANS:
             Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: s_store_id (type: string)
-              outputColumnNames: s_store_id
+              outputColumnNames: _col0
               Statistics: Num rows: 12 Data size: 25632 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: s_store_id (type: string), '0' (type: string)
+                keys: _col0 (type: string), '0' (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 24 Data size: 51264 Basic stats: COMPLETE Column stats: NONE


[33/35] hive git commit: HIVE-14990 : run all tests for MM tables and fix the issues that are found (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14990 : run all tests for MM tables and fix the issues that are found (Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 13ea883ce1f1a377b6ca07333f80912b75ed5534
Parents: a5ba17d
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Nov 17 19:49:58 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Nov 21 18:06:07 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/metadata/Hive.java    | 58 +++++++++++++++-----
 .../formatting/TextMetaDataFormatter.java       |  7 ++-
 2 files changed, 48 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/13ea883c/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 7dabd78..7514ec4 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
@@ -1601,7 +1601,8 @@ public class Hive {
         Utilities.LOG14535.info("maybe deleting stuff from " + oldPartPath + " (new " + newPartPath + ") for replace");
         if (replace && oldPartPath != null) {
           deleteOldPathForReplace(newPartPath, oldPartPath, getConf(),
-              new ValidWriteIds.IdPathFilter(mmWriteId, false, true), mmWriteId != null);
+              new ValidWriteIds.IdPathFilter(mmWriteId, false, true), mmWriteId != null,
+              tbl.isStoredAsSubDirectories() ? tbl.getSkewedColNames().size() : 0);
         }
       } else {
         // Either a non-MM query, or a load into MM table from an external source.
@@ -2080,7 +2081,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
       if (replace) {
         Path tableDest = tbl.getPath();
         deleteOldPathForReplace(tableDest, tableDest, sessionConf,
-            new ValidWriteIds.IdPathFilter(mmWriteId, false, true), mmWriteId != null);
+            new ValidWriteIds.IdPathFilter(mmWriteId, false, true), mmWriteId != null,
+            tbl.isStoredAsSubDirectories() ? tbl.getSkewedColNames().size() : 0);
       }
       newFiles = listFilesCreatedByQuery(loadPath, mmWriteId);
     } else {
@@ -3490,7 +3492,9 @@ private void constructOneLBLocationMap(FileStatus fSta,
       }
 
       if (oldPath != null) {
-        deleteOldPathForReplace(destf, oldPath, conf, deletePathFilter, isMmTable);
+        // TODO: we assume lbLevels is 0 here. Same as old code for non-MM.
+        //       For MM tables, this can only be a LOAD command. Does LOAD even support LB?
+        deleteOldPathForReplace(destf, oldPath, conf, deletePathFilter, isMmTable, 0);
       }
 
       // first call FileUtils.mkdir to make sure that destf directory exists, if not, it creates
@@ -3526,7 +3530,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
   }
 
   private void deleteOldPathForReplace(Path destPath, Path oldPath, HiveConf conf,
-      PathFilter pathFilter, boolean isMmTable) throws HiveException {
+      PathFilter pathFilter, boolean isMmTable, int lbLevels) throws HiveException {
     Utilities.LOG14535.info("Deleting old paths for replace in " + destPath + " and old path " + oldPath);
     boolean isOldPathUnderDestf = false;
     try {
@@ -3538,16 +3542,27 @@ private void constructOneLBLocationMap(FileStatus fSta,
       // not the destf or its subdir?
       isOldPathUnderDestf = isSubDir(oldPath, destPath, oldFs, destFs, false);
       if (isOldPathUnderDestf || isMmTable) {
-        FileStatus[] statuses = oldFs.listStatus(oldPath, pathFilter);
-        if (statuses == null || statuses.length == 0) return;
-        String s = "Deleting files under " + oldPath + " for replace: ";
-        for (FileStatus file : statuses) {
-          s += file.getPath().getName() + ", ";
-        }
-        Utilities.LOG14535.info(s);
-        if (!trashFiles(oldFs, statuses, conf)) {
-          throw new HiveException("Destination directory " + destPath
-              + " has not been cleaned up.");
+        if (lbLevels == 0 || !isMmTable) {
+          cleanUpOneDirectoryForReplace(oldPath, oldFs, pathFilter, conf);
+        } else {
+          // We need to clean up different MM IDs from each LB directory separately.
+          // Avoid temporary directories in the immediate table/part dir.
+          // TODO: we could just find directories with any MM directories inside?
+          //       the rest doesn't have to be cleaned up.
+          String mask = "[^._]*";
+          for (int i = 0; i < lbLevels - 1; ++i) {
+            mask += Path.SEPARATOR + "*";
+          }
+          Path glob = new Path(oldPath, mask);
+          FileStatus[] lbDirs = oldFs.globStatus(glob);
+          for (FileStatus lbDir : lbDirs) {
+            Path lbPath = lbDir.getPath();
+            if (!lbDir.isDirectory()) {
+              throw new HiveException("Unexpected path during overwrite: " + lbPath);
+            }
+            Utilities.LOG14535.info("Cleaning up LB directory " + lbPath);
+            cleanUpOneDirectoryForReplace(lbPath, oldFs, pathFilter, conf);
+          }
         }
       }
     } catch (IOException e) {
@@ -3563,6 +3578,21 @@ private void constructOneLBLocationMap(FileStatus fSta,
   }
 
 
+  private void cleanUpOneDirectoryForReplace(Path path, FileSystem fs,
+      PathFilter pathFilter, HiveConf conf) throws IOException, HiveException {
+    FileStatus[] statuses = fs.listStatus(path, pathFilter);
+    if (statuses == null || statuses.length == 0) return;
+    String s = "Deleting files under " + path + " for replace: ";
+    for (FileStatus file : statuses) {
+      s += file.getPath().getName() + ", ";
+    }
+    Utilities.LOG14535.info(s);
+    if (!trashFiles(fs, statuses, conf)) {
+      throw new HiveException("Old path " + path + " has not been cleaned up.");
+    }
+  }
+
+
   /**
    * Trashes or deletes all files under a directory. Leaves the directory as is.
    * @param fs FileSystem to use

http://git-wip-us.apache.org/repos/asf/hive/blob/13ea883c/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
index 22908d8..eac6963 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
@@ -318,10 +318,10 @@ class TextMetaDataFormatter implements MetaDataFormatter {
     if (!unknown) {
       for (Path loc : locations) {
         try {
-          FileStatus status = fs.getFileStatus(tblPath);
+          FileStatus status = fs.getFileStatus(loc);
           // no matter loc is the table location or part location, it must be a
           // directory.
-          if (!status.isDir()) {
+          if (!status.isDirectory()) {
             continue;
           }
           processDir(status, fs, fd);
@@ -381,6 +381,7 @@ class TextMetaDataFormatter implements MetaDataFormatter {
   }
 
   private void processDir(FileStatus status, FileSystem fs, FileData fd) throws IOException {
+    Utilities.LOG14535.info("Processing dir for status: " + status.getPath());
     long accessTime = status.getAccessTime();
     long updateTime = status.getModificationTime();
     if (accessTime > fd.lastAccessTime) {
@@ -391,7 +392,7 @@ class TextMetaDataFormatter implements MetaDataFormatter {
     }
     FileStatus[] files = fs.listStatus(status.getPath());
     for (FileStatus currentStatus : files) {
-      if (currentStatus.isDir()) {
+      if (currentStatus.isDirectory()) {
         processDir(currentStatus, fs, fd);
         continue;
       }


[20/35] hive git commit: HIVE-15178 : ORC stripe merge may produce many MR jobs and no merge if split size is small (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-15178 : ORC stripe merge may produce many MR jobs and no merge if split size is small (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/8a1dcd7e
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8a1dcd7e
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8a1dcd7e

Branch: refs/heads/hive-14535
Commit: 8a1dcd7e6c1befeb6f0fc38695cf24072570b2a5
Parents: 6efa869
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri Nov 18 11:06:39 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri Nov 18 11:06:49 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      | 42 +++++++++++---------
 .../hadoop/hive/ql/io/merge/MergeFileTask.java  |  4 ++
 2 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8a1dcd7e/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
index 9b07e21..34b683c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
@@ -260,25 +260,7 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
       throw new RuntimeException(e.getMessage(), e);
     }
 
-    if (mWork.getNumMapTasks() != null) {
-      job.setNumMapTasks(mWork.getNumMapTasks().intValue());
-    }
-
-    if (mWork.getMaxSplitSize() != null) {
-      HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, mWork.getMaxSplitSize().longValue());
-    }
-
-    if (mWork.getMinSplitSize() != null) {
-      HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZE, mWork.getMinSplitSize().longValue());
-    }
-
-    if (mWork.getMinSplitSizePerNode() != null) {
-      HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, mWork.getMinSplitSizePerNode().longValue());
-    }
-
-    if (mWork.getMinSplitSizePerRack() != null) {
-      HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, mWork.getMinSplitSizePerRack().longValue());
-    }
+    propagateSplitSettings(job, mWork);
 
     job.setNumReduceTasks(rWork != null ? rWork.getNumReduceTasks().intValue() : 0);
     job.setReducerClass(ExecReducer.class);
@@ -486,6 +468,28 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
     return (returnVal);
   }
 
+  public static void propagateSplitSettings(JobConf job, MapWork work) {
+    if (work.getNumMapTasks() != null) {
+      job.setNumMapTasks(work.getNumMapTasks().intValue());
+    }
+
+    if (work.getMaxSplitSize() != null) {
+      HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, work.getMaxSplitSize().longValue());
+    }
+
+    if (work.getMinSplitSize() != null) {
+      HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZE, work.getMinSplitSize().longValue());
+    }
+
+    if (work.getMinSplitSizePerNode() != null) {
+      HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, work.getMinSplitSizePerNode().longValue());
+    }
+
+    if (work.getMinSplitSizePerRack() != null) {
+      HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, work.getMinSplitSizePerRack().longValue());
+    }
+  }
+
   private void handleSampling(Context context, MapWork mWork, JobConf job)
       throws Exception {
     assert mWork.getAliasToWork().keySet().size() == 1;

http://git-wip-us.apache.org/repos/asf/hive/blob/8a1dcd7e/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java b/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java
index 67a6dc7..b3b6431 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.io.merge;
 
+import org.apache.hadoop.hive.ql.exec.mr.ExecDriver;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -108,6 +110,8 @@ public class MergeFileTask extends Task<MergeFileWork> implements Serializable,
         fs.mkdirs(tempOutPath);
       }
 
+      ExecDriver.propagateSplitSettings(job, work);
+
       // set job name
       boolean noName = StringUtils.isEmpty(job.get(MRJobConfig.JOB_NAME));
 


[27/35] hive git commit: HIVE-13557: Make interval keyword optional while specifying DAY in interval arithmetic (Zoltan Haindrich reviewed by Pengcheng Xiong)

Posted by se...@apache.org.
HIVE-13557: Make interval keyword optional while specifying DAY in interval arithmetic (Zoltan Haindrich reviewed by Pengcheng Xiong)


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

Branch: refs/heads/hive-14535
Commit: e2a6273afc4dc2a98a9ab238753daf268295fd9f
Parents: 678e6ca
Author: Pengcheng Xiong <px...@apache.org>
Authored: Fri Nov 18 16:02:09 2016 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Fri Nov 18 16:02:09 2016 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   2 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |  14 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  18 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |  82 -----
 .../udf/generic/GenericUDFInternalInterval.java | 322 +++++++++++++++++++
 .../hive/ql/parse/TestParseDriverIntervals.java |  83 +++++
 .../generic/TestGenericUDFInternalInterval.java | 144 +++++++++
 .../test/queries/clientpositive/interval_alt.q  |  36 +++
 .../results/clientpositive/interval_alt.q.out   | 135 ++++++++
 .../clientpositive/llap/vector_interval_1.q.out |   6 +-
 .../llap/vector_interval_arithmetic.q.out       |  17 +-
 .../results/clientpositive/show_functions.q.out |   1 +
 .../clientpositive/vector_interval_1.q.out      |   6 +-
 .../vector_interval_arithmetic.q.out            |  17 +-
 14 files changed, 763 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 0dbbc1d..83d78bc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -468,6 +468,8 @@ public final class FunctionRegistry {
     system.registerGenericUDF("unix_timestamp", GenericUDFUnixTimeStamp.class);
     system.registerGenericUDF("to_unix_timestamp", GenericUDFToUnixTimeStamp.class);
 
+    system.registerGenericUDF("internal_interval", GenericUDFInternalInterval.class);
+
     // Generic UDTF's
     system.registerGenericUDTF("explode", GenericUDTFExplode.class);
     system.registerGenericUDTF("replicate_rows", GenericUDTFReplicateRows.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index 4357328..e9ccfd2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -303,15 +303,15 @@ KW_AUTHORIZATION: 'AUTHORIZATION';
 KW_CONF: 'CONF';
 KW_VALUES: 'VALUES';
 KW_RELOAD: 'RELOAD';
-KW_YEAR: 'YEAR';
+KW_YEAR: 'YEAR' | 'YEARS';
 KW_QUARTER: 'QUARTER';
-KW_MONTH: 'MONTH';
-KW_WEEK: 'WEEK';
-KW_DAY: 'DAY';
+KW_MONTH: 'MONTH' | 'MONTHS';
+KW_WEEK: 'WEEK' | 'WEEKS';
+KW_DAY: 'DAY' | 'DAYS';
 KW_DOW: 'DAYOFWEEK';
-KW_HOUR: 'HOUR';
-KW_MINUTE: 'MINUTE';
-KW_SECOND: 'SECOND';
+KW_HOUR: 'HOUR' | 'HOURS';
+KW_MINUTE: 'MINUTE' | 'MINUTES';
+KW_SECOND: 'SECOND' | 'SECONDS';
 KW_START: 'START';
 KW_TRANSACTION: 'TRANSACTION';
 KW_COMMIT: 'COMMIT';

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index 89dad86..1868dda 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -314,11 +314,9 @@ timeQualifiers
 constant
 @init { gParent.pushMsg("constant", state); }
 @after { gParent.popMsg(state); }
-    :
-    Number
+    : Number
     | dateLiteral
     | timestampLiteral
-    | intervalLiteral
     | StringLiteral
     | stringLiteralSequence
     | IntegralLiteral
@@ -361,12 +359,13 @@ timestampLiteral
     KW_CURRENT_TIMESTAMP -> ^(TOK_FUNCTION KW_CURRENT_TIMESTAMP)
     ;
 
-intervalLiteral
+intervalExpression
     :
-    KW_INTERVAL StringLiteral qualifiers=intervalQualifiers ->
-    {
-      adaptor.create(((CommonTree)qualifiers.getTree()).getType(), $StringLiteral.text)
-    }
+    KW_INTERVAL? k1=(StringLiteral|Number) q1=intervalQualifiers ->
+		^(TOK_FUNCTION Identifier["internal_interval"] NumberLiteral[Integer.toString(((CommonTree)q1.getTree()).token.getType())] $k1)
+    |
+    KW_INTERVAL? LPAREN k2=expression RPAREN q2=intervalQualifiers ->
+		^(TOK_FUNCTION Identifier["internal_interval"] NumberLiteral[Integer.toString(((CommonTree)q2.getTree()).token.getType())] $k2)
     ;
 
 intervalQualifiers
@@ -391,6 +390,7 @@ expression
 atomExpression
     :
     (KW_NULL) => KW_NULL -> TOK_NULL
+    | (intervalExpression)=>intervalExpression
     | (constant) => constant
     | castExpression
     | extractExpression
@@ -770,4 +770,4 @@ nonReserved
 sql11ReservedKeywordsUsedAsFunctionName
     :
     KW_BIGINT | KW_BINARY | KW_BOOLEAN | KW_CURRENT_DATE | KW_CURRENT_TIMESTAMP | KW_DATE | KW_DOUBLE | KW_FLOAT | KW_INT | KW_SMALLINT | KW_TIMESTAMP
-    ;
\ No newline at end of file
+    ;

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 5e708d3..96dab3b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -191,15 +191,6 @@ public class TypeCheckProcFactory {
         + HiveParser.KW_FALSE + "%"), tf.getBoolExprProcessor());
     opRules.put(new RuleRegExp("R5", HiveParser.TOK_DATELITERAL + "%|"
         + HiveParser.TOK_TIMESTAMPLITERAL + "%"), tf.getDateTimeExprProcessor());
-    opRules.put(new RuleRegExp("R6",
-        HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_YEAR_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_MONTH_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_DAY_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_HOUR_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_MINUTE_LITERAL + "%|"
-        + HiveParser.TOK_INTERVAL_SECOND_LITERAL + "%"), tf.getIntervalExprProcessor());
     opRules.put(new RuleRegExp("R7", HiveParser.TOK_TABLE_OR_COL + "%"),
         tf.getColumnExprProcessor());
     opRules.put(new RuleRegExp("R8", HiveParser.TOK_SUBQUERY_OP + "%"),
@@ -512,79 +503,6 @@ public class TypeCheckProcFactory {
   }
 
   /**
-   * Processor for interval constants.
-   */
-  public static class IntervalExprProcessor implements NodeProcessor {
-
-    private static final BigDecimal NANOS_PER_SEC_BD = new BigDecimal(DateUtils.NANOS_PER_SEC);
-    @Override
-    public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
-        Object... nodeOutputs) throws SemanticException {
-
-      TypeCheckCtx ctx = (TypeCheckCtx) procCtx;
-      if (ctx.getError() != null) {
-        return null;
-      }
-
-      ExprNodeDesc desc = TypeCheckProcFactory.processGByExpr(nd, procCtx);
-      if (desc != null) {
-        return desc;
-      }
-
-      ASTNode expr = (ASTNode) nd;
-      String intervalString = BaseSemanticAnalyzer.stripQuotes(expr.getText());
-
-      // Get the string value and convert to a Interval value.
-      try {
-        switch (expr.getType()) {
-          case HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo,
-                HiveIntervalYearMonth.valueOf(intervalString));
-          case HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                HiveIntervalDayTime.valueOf(intervalString));
-          case HiveParser.TOK_INTERVAL_YEAR_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo,
-                new HiveIntervalYearMonth(Integer.parseInt(intervalString), 0));
-          case HiveParser.TOK_INTERVAL_MONTH_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo,
-                new HiveIntervalYearMonth(0, Integer.parseInt(intervalString)));
-          case HiveParser.TOK_INTERVAL_DAY_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                new HiveIntervalDayTime(Integer.parseInt(intervalString), 0, 0, 0, 0));
-          case HiveParser.TOK_INTERVAL_HOUR_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                new HiveIntervalDayTime(0, Integer.parseInt(intervalString), 0, 0, 0));
-          case HiveParser.TOK_INTERVAL_MINUTE_LITERAL:
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                new HiveIntervalDayTime(0, 0, Integer.parseInt(intervalString), 0, 0));
-          case HiveParser.TOK_INTERVAL_SECOND_LITERAL:
-            BigDecimal bd = new BigDecimal(intervalString);
-            BigDecimal bdSeconds = new BigDecimal(bd.toBigInteger());
-            BigDecimal bdNanos = bd.subtract(bdSeconds);
-            return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
-                new HiveIntervalDayTime(0, 0, 0, bdSeconds.intValueExact(),
-                    bdNanos.multiply(NANOS_PER_SEC_BD).intValue()));
-          default:
-            throw new IllegalArgumentException("Invalid time literal type " + expr.getType());
-        }
-      } catch (Exception err) {
-        throw new SemanticException(
-            "Unable to convert interval literal '" + intervalString + "' to interval value.", err);
-      }
-    }
-  }
-
-  /**
-   * Factory method to get IntervalExprProcessor.
-   *
-   * @return IntervalExprProcessor.
-   */
-  public IntervalExprProcessor getIntervalExprProcessor() {
-    return new IntervalExprProcessor();
-  }
-
-  /**
    * Processor for table columns.
    */
   public static class ColumnExprProcessor implements NodeProcessor {

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInternalInterval.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInternalInterval.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInternalInterval.java
new file mode 100644
index 0000000..fcf291a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFInternalInterval.java
@@ -0,0 +1,322 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import java.math.BigDecimal;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils.PrimitiveGrouping;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hive.common.util.DateUtils;
+
+/**
+ * GenericUDF Class for support of "INTERVAL (expression) (DAY|YEAR|...)".
+ */
+@Description(name = "internal_interval", value = "_FUNC_(intervalType,intervalArg)",
+  extended = "this method is not designed to be used by directly calling it - it provides internal support for 'INTERVAL (intervalArg) intervalType' constructs")
+
+public class GenericUDFInternalInterval extends GenericUDF {
+
+  private static Map<Integer, IntervalProcessor> processorMap;
+
+  private transient IntervalProcessor processor;
+  private transient PrimitiveObjectInspector inputOI;
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+
+    // read operation mode
+    if (!(arguments[0] instanceof ConstantObjectInspector)) {
+      throw new UDFArgumentTypeException(0,
+          getFuncName() + ": may only accept constant as first argument");
+    }
+    Integer operationMode = getConstantIntValue(arguments, 0);
+    if (operationMode == null) {
+      throw new UDFArgumentTypeException(0, "must supply operationmode");
+    }
+
+    processor = getProcessorMap().get(operationMode);
+    if (processor == null) {
+      throw new UDFArgumentTypeException(0,
+          getFuncName() + ": unsupported operationMode: " + operationMode);
+    }
+
+    // check value argument
+    if (arguments[1].getCategory() != Category.PRIMITIVE) {
+      throw new UDFArgumentTypeException(1,
+          "The first argument to " + getFuncName() + " must be primitive");
+    }
+
+    inputOI = (PrimitiveObjectInspector) arguments[1];
+
+    PrimitiveCategory inputCategory = inputOI.getPrimitiveCategory();
+
+    if (!isValidInputCategory(inputCategory)) {
+      throw new UDFArgumentTypeException(1, "The second argument to " + getFuncName()
+          + " must be from the string group or numeric group (except:float/double)");
+    }
+
+    if (arguments[1] instanceof ConstantObjectInspector) {
+      // return value as constant in case arg is constant
+      return PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+          processor.getTypeInfo(), processor.evaluate(getConstantStringValue(arguments, 1)));
+    } else {
+      return PrimitiveObjectInspectorFactory
+          .getPrimitiveWritableObjectInspector(processor.getTypeInfo());
+    }
+  }
+
+  private boolean isValidInputCategory(PrimitiveCategory cat) throws UDFArgumentTypeException {
+    PrimitiveGrouping inputOIGroup = PrimitiveObjectInspectorUtils.getPrimitiveGrouping(cat);
+
+    if (inputOIGroup == PrimitiveGrouping.STRING_GROUP) {
+      return true;
+    }
+    if (inputOIGroup == PrimitiveGrouping.NUMERIC_GROUP) {
+      switch (cat) {
+      case DOUBLE:
+      case FLOAT:
+        return false;
+      default:
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    String argString = PrimitiveObjectInspectorUtils.getString(arguments[1].get(), inputOI);
+    if (argString == null) {
+      return null;
+    }
+    try {
+      return processor.evaluate(argString);
+    } catch (Exception e) {
+      throw new UDFArgumentTypeException(1, "Error parsing interval " + argString + " using:"
+          + processor.getClass().getSimpleName());
+    }
+  }
+
+  private static interface IntervalProcessor {
+
+    Integer getKey();
+
+    PrimitiveTypeInfo getTypeInfo();
+
+    Object evaluate(String arg) throws UDFArgumentException;
+  }
+
+  private static abstract class AbstractDayTimeIntervalProcessor implements IntervalProcessor {
+    private transient HiveIntervalDayTimeWritable intervalResult =
+        new HiveIntervalDayTimeWritable();
+
+    @Override
+    public final PrimitiveTypeInfo getTypeInfo() {
+      return TypeInfoFactory.intervalDayTimeTypeInfo;
+    }
+
+    @Override
+    public final Object evaluate(String arg) throws UDFArgumentException {
+      intervalResult.set(getIntervalDayTime(arg));
+      return intervalResult;
+    }
+
+    abstract protected HiveIntervalDayTime getIntervalDayTime(String arg);
+  }
+
+  private static abstract class AbstractYearMonthIntervalProcessor implements IntervalProcessor {
+    private transient HiveIntervalYearMonthWritable intervalResult =
+        new HiveIntervalYearMonthWritable();
+
+    @Override
+    public final PrimitiveTypeInfo getTypeInfo() {
+      return TypeInfoFactory.intervalYearMonthTypeInfo;
+    }
+
+    @Override
+    public final Object evaluate(String arg) throws UDFArgumentException {
+      intervalResult.set(getIntervalYearMonth(arg));
+      return intervalResult;
+    }
+
+    abstract protected HiveIntervalYearMonth getIntervalYearMonth(String arg);
+  }
+
+  private static class IntervalDayLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_DAY_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      return new HiveIntervalDayTime(Integer.parseInt(arg), 0, 0, 0, 0);
+    }
+  }
+
+  private static class IntervalHourLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_HOUR_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      return new HiveIntervalDayTime(0, Integer.parseInt(arg), 0, 0, 0);
+    }
+  }
+
+  private static class IntervalMinuteLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_MINUTE_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      return new HiveIntervalDayTime(0, 0, Integer.parseInt(arg), 0, 0);
+    }
+  }
+
+  private static class IntervalSecondLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+
+    private static final BigDecimal NANOS_PER_SEC_BD = new BigDecimal(DateUtils.NANOS_PER_SEC);
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_SECOND_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      BigDecimal bd = new BigDecimal(arg);
+      BigDecimal bdSeconds = new BigDecimal(bd.toBigInteger());
+      BigDecimal bdNanos = bd.subtract(bdSeconds);
+      return new HiveIntervalDayTime(0, 0, 0, bdSeconds.intValueExact(),
+          bdNanos.multiply(NANOS_PER_SEC_BD).intValue());
+    }
+  }
+
+  private static class IntervalDayTimeLiteralProcessor extends AbstractDayTimeIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalDayTime getIntervalDayTime(String arg) {
+      return HiveIntervalDayTime.valueOf(arg);
+    }
+  }
+
+  private static class IntervalYearMonthLiteralProcessor
+      extends AbstractYearMonthIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalYearMonth getIntervalYearMonth(String arg) {
+      return HiveIntervalYearMonth.valueOf(arg);
+    }
+  }
+
+  private static class IntervalYearLiteralProcessor extends AbstractYearMonthIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_YEAR_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalYearMonth getIntervalYearMonth(String arg) {
+      return new HiveIntervalYearMonth(Integer.parseInt(arg), 0);
+    }
+  }
+
+  private static class IntervalMonthLiteralProcessor extends AbstractYearMonthIntervalProcessor {
+
+    @Override
+    public Integer getKey() {
+      return HiveParser.TOK_INTERVAL_MONTH_LITERAL;
+    }
+
+    @Override
+    protected HiveIntervalYearMonth getIntervalYearMonth(String arg) {
+      return new HiveIntervalYearMonth(0, Integer.parseInt(arg));
+    }
+  }
+
+  private static Map<Integer, IntervalProcessor> getProcessorMap() {
+
+    if (processorMap != null) {
+      return processorMap;
+    }
+
+    Map<Integer, IntervalProcessor> ret = new HashMap<>();
+    IntervalProcessor ips[]=new IntervalProcessor[]{
+        new IntervalDayTimeLiteralProcessor(),
+
+        new IntervalDayLiteralProcessor(),
+        new IntervalHourLiteralProcessor(),
+        new IntervalMinuteLiteralProcessor(),
+        new IntervalSecondLiteralProcessor(),
+
+        new IntervalYearMonthLiteralProcessor(),
+
+        new IntervalYearLiteralProcessor(),
+        new IntervalMonthLiteralProcessor(),
+    };
+
+    for (IntervalProcessor ip : ips) {
+      ret.put(ip.getKey(), ip);
+    }
+
+    return processorMap=ret;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return String.format("%s(%s)", processor.getClass().getSimpleName(), children[1]);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestParseDriverIntervals.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestParseDriverIntervals.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestParseDriverIntervals.java
new file mode 100644
index 0000000..e5db412
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestParseDriverIntervals.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse;
+
+import static org.junit.Assert.assertNotNull;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestParseDriverIntervals {
+
+  private String query;
+  private ParseDriver parseDriver;
+
+  @Parameters(name = "{0}")
+  public static List<Object[]> getParameters() {
+    List<Object[]> ret = new ArrayList<>();
+    ret.add(new Object[] { "select 1 days" });
+    ret.add(new Object[] { "select (1) days" });
+    ret.add(new Object[] { "select (1) day" });
+    ret.add(new Object[] { "select (1+1) days" });
+    ret.add(new Object[] { "select interval 1 days" });
+    ret.add(new Object[] { "select interval '1' days" });
+    ret.add(new Object[] { "select interval (x) days" });
+    ret.add(new Object[] { "select interval (x+1) days" });
+    ret.add(new Object[] { "select interval (1+x) days" });
+    ret.add(new Object[] { "select interval (1+1) days" });
+    ret.add(new Object[] { "select interval (x+1) days" });
+
+    return ret;
+  }
+
+  public TestParseDriverIntervals(String query) {
+    parseDriver = new ParseDriver();
+    this.query = query;
+  }
+
+  @Test
+  public void parseInterval() throws Exception {
+    ASTNode root = parseDriver.parse(query);
+    assertNotNull("failed: " + query, findFunctionNode(root));
+    System.out.println(root.dump());
+  }
+
+  private ASTNode findFunctionNode(ASTNode n) {
+    if (n.getType() == HiveParser.TOK_FUNCTION) {
+      if ("internal_interval".equals(n.getChild(0).getText())) {
+        return n;
+      }
+    }
+    ArrayList<Node> children = n.getChildren();
+    if (children != null) {
+      for (Node c : children) {
+        ASTNode r = findFunctionNode((ASTNode) c);
+        if (r != null)
+          return r;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInternalInterval.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInternalInterval.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInternalInterval.java
new file mode 100644
index 0000000..8166bc0
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDFInternalInterval.java
@@ -0,0 +1,144 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.udf.generic;
+
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestGenericUDFInternalInterval {
+
+  @Test
+  public void testDayInterval() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs =
+          {
+              PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+                  TypeInfoFactory.intTypeInfo,
+                  new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+              PrimitiveObjectInspectorFactory.writableStringObjectInspector
+          };
+
+      DeferredObject[] args = {
+          new DeferredJavaObject(new ByteWritable((byte) 4)),
+          new DeferredJavaObject(new Text("8"))
+          };
+
+      PrimitiveObjectInspector oi = (PrimitiveObjectInspector) udf.initialize(inputOIs);
+      Assert.assertEquals(TypeInfoFactory.intervalDayTimeTypeInfo, oi.getTypeInfo());
+      HiveIntervalDayTimeWritable res = (HiveIntervalDayTimeWritable) udf.evaluate(args);
+      Assert.assertEquals(8, res.getHiveIntervalDayTime().getDays());
+    }
+  }
+
+  @Test
+  public void testDayIntervalConstant() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs = {
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo, new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo, new IntWritable(3))
+          };
+
+      PrimitiveObjectInspector oi = (PrimitiveObjectInspector) udf.initialize(inputOIs);
+      Assert.assertEquals(TypeInfoFactory.intervalDayTimeTypeInfo, oi.getTypeInfo());
+      ConstantObjectInspector coi = (ConstantObjectInspector) oi;
+      HiveIntervalDayTimeWritable res =
+          (HiveIntervalDayTimeWritable) coi.getWritableConstantValue();
+      Assert.assertEquals(3, res.getHiveIntervalDayTime().getDays());
+    }
+  }
+
+  @Test(expected = UDFArgumentException.class)
+  public void testDoubleArgumentIsNotSupported() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs = {
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo,
+              new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+          PrimitiveObjectInspectorFactory.writableDoubleObjectInspector,
+          };
+
+      // should detect double
+      udf.initialize(inputOIs);
+    }
+  }
+
+  @Test(expected = UDFArgumentException.class)
+  public void testInvalidString() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs = {
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo,
+              new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+          PrimitiveObjectInspectorFactory.writableStringObjectInspector,
+          };
+
+      DeferredObject[] args = {
+          new DeferredJavaObject(new ByteWritable((byte) 4)),
+          new DeferredJavaObject(new Text("invalid"))
+          };
+
+      PrimitiveObjectInspector oi = (PrimitiveObjectInspector) udf.initialize(inputOIs);
+      Assert.assertEquals(TypeInfoFactory.intervalDayTimeTypeInfo, oi.getTypeInfo());
+      // should emit an exception
+      udf.evaluate(args);
+    }
+  }
+
+  @Test
+  public void testNullBypass() throws Exception {
+    try (GenericUDFInternalInterval udf = new GenericUDFInternalInterval()) {
+
+      ObjectInspector[] inputOIs = {
+          PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+              TypeInfoFactory.intTypeInfo,
+              new IntWritable(HiveParser.TOK_INTERVAL_DAY_LITERAL)),
+          PrimitiveObjectInspectorFactory.writableStringObjectInspector
+          };
+      DeferredObject[] args = {
+              new DeferredJavaObject(new ByteWritable((byte) 4)),
+              new DeferredJavaObject(null)
+              };
+
+      PrimitiveObjectInspector oi = (PrimitiveObjectInspector) udf.initialize(inputOIs);
+      Assert.assertEquals(TypeInfoFactory.intervalDayTimeTypeInfo, oi.getTypeInfo());
+      HiveIntervalDayTimeWritable res = (HiveIntervalDayTimeWritable) udf.evaluate(args);
+      Assert.assertEquals(null, res);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/queries/clientpositive/interval_alt.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/interval_alt.q b/ql/src/test/queries/clientpositive/interval_alt.q
new file mode 100644
index 0000000..a4fbe9f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/interval_alt.q
@@ -0,0 +1,36 @@
+
+select
+	(1) second,
+	 2  seconds,
+	(1) minute,
+	 2  minutes,
+	(1) hour,
+	 2  hours,
+	(1) day,
+	 2  days,
+	(1) month,
+	 2  months,
+	(1) year,
+	 2  years;
+
+select date '2012-01-01' + 30 days;
+select date '2012-01-01' - 30 days;
+
+create table t (dt int);
+insert into t values (1),(2);
+
+-- expressions/columnref
+explain
+select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t;
+
+select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t;

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/interval_alt.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/interval_alt.q.out b/ql/src/test/results/clientpositive/interval_alt.q.out
new file mode 100644
index 0000000..eb4d10b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/interval_alt.q.out
@@ -0,0 +1,135 @@
+PREHOOK: query: select
+	(1) second,
+	 2  seconds,
+	(1) minute,
+	 2  minutes,
+	(1) hour,
+	 2  hours,
+	(1) day,
+	 2  days,
+	(1) month,
+	 2  months,
+	(1) year,
+	 2  years
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select
+	(1) second,
+	 2  seconds,
+	(1) minute,
+	 2  minutes,
+	(1) hour,
+	 2  hours,
+	(1) day,
+	 2  days,
+	(1) month,
+	 2  months,
+	(1) year,
+	 2  years
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+0 00:00:01.000000000	0 00:00:02.000000000	0 00:01:00.000000000	0 00:02:00.000000000	0 01:00:00.000000000	0 02:00:00.000000000	1 00:00:00.000000000	2 00:00:00.000000000	0-1	0-2	1-0	2-0
+PREHOOK: query: select date '2012-01-01' + 30 days
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select date '2012-01-01' + 30 days
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+2012-01-31 00:00:00
+PREHOOK: query: select date '2012-01-01' - 30 days
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select date '2012-01-01' - 30 days
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+2011-12-02 00:00:00
+PREHOOK: query: create table t (dt int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t
+POSTHOOK: query: create table t (dt int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t
+PREHOOK: query: insert into t values (1),(2)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@t
+POSTHOOK: query: insert into t values (1),(2)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@t
+POSTHOOK: Lineage: t.dt EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: -- expressions/columnref
+explain
+select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t
+PREHOOK: type: QUERY
+POSTHOOK: query: -- expressions/columnref
+explain
+select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t
+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: t
+            Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: (2012-01-01 + IntervalDayLiteralProcessor(((- dt) * dt))) (type: timestamp), (2012-01-01 - IntervalDayLiteralProcessor(((- dt) * dt))) (type: timestamp), 2012-01-04 (type: date), (2012-01-01 + IntervalYearMonthLiteralProcessor(concat(dt, '-1'))) (type: date)
+              outputColumnNames: _col0, _col1, _col2, _col3
+              Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE Column stats: NONE
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: select
+	date '2012-01-01' +          (-dt*dt) day,
+	date '2012-01-01' - interval (-dt*dt) day,
+	date '2012-01-01' + 1 day + '2' days,
+	date '2012-01-01' + (dt || '-1') year to month
+	from t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+2011-12-31 00:00:00	2012-01-02 00:00:00	2012-01-04	2013-02-01
+2011-12-28 00:00:00	2012-01-05 00:00:00	2012-01-04	2014-02-01

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out b/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out
index d8003ba..0dd8fc5 100644
--- a/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_interval_1.q.out
@@ -86,7 +86,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: string), 1-2 (type: interval_year_month), VALUE._col0 (type: interval_year_month), 1 02:03:04.000000000 (type: interval_day_time), VALUE._col1 (type: interval_day_time)
+                expressions: KEY.reducesinkkey0 (type: string), 1-2 (type: interval_year_month), VALUE._col1 (type: interval_year_month), 1 02:03:04.000000000 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
@@ -177,7 +177,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col0 (type: interval_year_month), VALUE._col1 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col2 (type: interval_year_month), VALUE._col3 (type: interval_year_month)
+                expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col1 (type: interval_year_month), VALUE._col2 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col4 (type: interval_year_month), VALUE._col5 (type: interval_year_month)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
@@ -274,7 +274,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: date), 2 04:06:08.000000000 (type: interval_day_time), VALUE._col1 (type: interval_day_time), VALUE._col2 (type: interval_day_time), 0 00:00:00.000000000 (type: interval_day_time), VALUE._col4 (type: interval_day_time), VALUE._col5 (type: interval_day_time)
+                expressions: KEY.reducesinkkey0 (type: date), 2 04:06:08.000000000 (type: interval_day_time), VALUE._col0 (type: interval_day_time), VALUE._col1 (type: interval_day_time), 0 00:00:00.000000000 (type: interval_day_time), VALUE._col2 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out b/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out
index 13a8b35..8e3bc39 100644
--- a/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_interval_arithmetic.q.out
@@ -137,7 +137,7 @@ order by dateval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-dateval	c1	c2	c3	c4	c5	c6
+dateval	_c1	_c2	_c3	_c4	_c5	_c6
 0004-09-22	0002-07-22	0006-11-22	0006-11-22	0002-07-22	0002-07-22	0006-11-22
 0528-10-27	0526-08-27	0530-12-27	0530-12-27	0526-08-27	0526-08-27	0530-12-27
 1319-02-02	1316-12-02	1321-04-02	1321-04-02	1316-12-02	1316-12-02	1321-04-02
@@ -427,7 +427,7 @@ order by tsval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-tsval	c1	c2	c3	c4	c5	c6
+tsval	_c1	_c2	_c3	_c4	_c5	_c6
 0004-09-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0006-11-22 18:26:29.519542222	0006-11-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0006-11-22 18:26:29.519542222
 0528-10-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0530-12-27 08:15:18.941718273	0530-12-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0530-12-27 08:15:18.941718273
 1319-02-02 16:31:57.778	1316-12-02 16:31:57.778	1321-04-02 16:31:57.778	1321-04-02 16:31:57.778	1316-12-02 16:31:57.778	1316-12-02 16:31:57.778	1321-04-02 16:31:57.778
@@ -515,7 +515,8 @@ STAGE PLANS:
                   Select Operator
                     Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
-                      sort order: 
+                      key expressions: 5-5 (type: interval_year_month)
+                      sort order: +
                       Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                       TopN Hash Memory Usage: 0.1
             Execution mode: vectorized, llap
@@ -524,7 +525,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: 5-5 (type: interval_year_month), -1-1 (type: interval_year_month)
+                expressions: KEY.reducesinkkey0 (type: interval_year_month), -1-1 (type: interval_year_month)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
@@ -562,7 +563,7 @@ limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-c0	c1
+_c0	_c1
 5-5	-1-1
 5-5	-1-1
 PREHOOK: query: -- interval day-time arithmetic
@@ -667,7 +668,7 @@ order by dateval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-dateval	_c1	_c2	_c3	_c4	_c5	_c6
+dateval	c1	c2	c3	c4	c5	c6
 0004-09-22	0004-06-14 12:37:26.876543211	0004-12-30 11:22:33.123456789	0004-12-30 11:22:33.123456789	0004-06-14 12:37:26.876543211	0004-06-14 12:37:26.876543211	0004-12-30 11:22:33.123456789
 0528-10-27	0528-07-19 12:37:26.876543211	0529-02-03 11:22:33.123456789	0529-02-03 11:22:33.123456789	0528-07-19 12:37:26.876543211	0528-07-19 12:37:26.876543211	0529-02-03 11:22:33.123456789
 1319-02-02	1318-10-25 12:37:26.876543211	1319-05-12 11:22:33.123456789	1319-05-12 11:22:33.123456789	1318-10-25 12:37:26.876543211	1318-10-25 12:37:26.876543211	1319-05-12 11:22:33.123456789
@@ -961,7 +962,7 @@ order by tsval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-tsval	_c1	_c2	_c3	_c4	_c5	_c6
+tsval	c1	c2	c3	c4	c5	c6
 0004-09-22 18:26:29.519542222	0004-06-15 07:03:56.396085433	0004-12-31 05:49:02.642999011	0004-12-31 05:49:02.642999011	0004-06-15 07:03:56.396085433	0004-06-15 07:03:56.396085433	0004-12-31 05:49:02.642999011
 0528-10-27 08:15:18.941718273	0528-07-19 20:52:45.818261484	0529-02-03 19:37:52.065175062	0529-02-03 19:37:52.065175062	0528-07-19 20:52:45.818261484	0528-07-19 20:52:45.818261484	0529-02-03 19:37:52.065175062
 1319-02-02 16:31:57.778	1318-10-26 05:09:24.654543211	1319-05-13 03:54:30.901456789	1319-05-13 03:54:30.901456789	1318-10-26 05:09:24.654543211	1318-10-26 05:09:24.654543211	1319-05-13 03:54:30.901456789
@@ -1080,7 +1081,7 @@ limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-_c0	_c1
+c0	c1
 109 20:30:40.246913578	89 02:14:26.000000000
 109 20:30:40.246913578	89 02:14:26.000000000
 PREHOOK: query: drop table interval_arithmetic_1

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index d2f3b97..6cb9015 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -113,6 +113,7 @@ index
 initcap
 inline
 instr
+internal_interval
 isnotnull
 isnull
 java_method

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/vector_interval_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_interval_1.q.out b/ql/src/test/results/clientpositive/vector_interval_1.q.out
index 373a6de..f18e56c 100644
--- a/ql/src/test/results/clientpositive/vector_interval_1.q.out
+++ b/ql/src/test/results/clientpositive/vector_interval_1.q.out
@@ -77,7 +77,7 @@ STAGE PLANS:
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), 1-2 (type: interval_year_month), VALUE._col0 (type: interval_year_month), 1 02:03:04.000000000 (type: interval_day_time), VALUE._col1 (type: interval_day_time)
+          expressions: KEY.reducesinkkey0 (type: string), 1-2 (type: interval_year_month), VALUE._col1 (type: interval_year_month), 1 02:03:04.000000000 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
@@ -159,7 +159,7 @@ STAGE PLANS:
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col0 (type: interval_year_month), VALUE._col1 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col2 (type: interval_year_month), VALUE._col3 (type: interval_year_month)
+          expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col1 (type: interval_year_month), VALUE._col2 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col4 (type: interval_year_month), VALUE._col5 (type: interval_year_month)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
           Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
@@ -247,7 +247,7 @@ STAGE PLANS:
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: date), 2 04:06:08.000000000 (type: interval_day_time), VALUE._col1 (type: interval_day_time), VALUE._col2 (type: interval_day_time), 0 00:00:00.000000000 (type: interval_day_time), VALUE._col4 (type: interval_day_time), VALUE._col5 (type: interval_day_time)
+          expressions: KEY.reducesinkkey0 (type: date), 2 04:06:08.000000000 (type: interval_day_time), VALUE._col0 (type: interval_day_time), VALUE._col1 (type: interval_day_time), 0 00:00:00.000000000 (type: interval_day_time), VALUE._col2 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
           Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE
           File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/e2a6273a/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out b/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
index ff16b3b..5d1dca6 100644
--- a/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
+++ b/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
@@ -128,7 +128,7 @@ order by dateval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-dateval	c1	c2	c3	c4	c5	c6
+dateval	_c1	_c2	_c3	_c4	_c5	_c6
 0004-09-22	0002-07-22	0006-11-22	0006-11-22	0002-07-22	0002-07-22	0006-11-22
 0528-10-27	0526-08-27	0530-12-27	0530-12-27	0526-08-27	0526-08-27	0530-12-27
 1319-02-02	1316-12-02	1321-04-02	1321-04-02	1316-12-02	1316-12-02	1321-04-02
@@ -400,7 +400,7 @@ order by tsval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-tsval	c1	c2	c3	c4	c5	c6
+tsval	_c1	_c2	_c3	_c4	_c5	_c6
 0004-09-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0006-11-22 18:26:29.519542222	0006-11-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0002-07-22 18:26:29.519542222	0006-11-22 18:26:29.519542222
 0528-10-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0530-12-27 08:15:18.941718273	0530-12-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0526-08-27 08:15:18.941718273	0530-12-27 08:15:18.941718273
 1319-02-02 16:31:57.778	1316-12-02 16:31:57.778	1321-04-02 16:31:57.778	1321-04-02 16:31:57.778	1316-12-02 16:31:57.778	1316-12-02 16:31:57.778	1321-04-02 16:31:57.778
@@ -482,13 +482,14 @@ STAGE PLANS:
             Select Operator
               Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
-                sort order: 
+                key expressions: 5-5 (type: interval_year_month)
+                sort order: +
                 Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
                 TopN Hash Memory Usage: 0.1
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
-          expressions: 5-5 (type: interval_year_month), -1-1 (type: interval_year_month)
+          expressions: KEY.reducesinkkey0 (type: interval_year_month), -1-1 (type: interval_year_month)
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 50 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
           Limit
@@ -526,7 +527,7 @@ limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-c0	c1
+_c0	_c1
 5-5	-1-1
 5-5	-1-1
 PREHOOK: query: -- interval day-time arithmetic
@@ -622,7 +623,7 @@ order by dateval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-dateval	_c1	_c2	_c3	_c4	_c5	_c6
+dateval	c1	c2	c3	c4	c5	c6
 0004-09-22	0004-06-14 12:37:26.876543211	0004-12-30 11:22:33.123456789	0004-12-30 11:22:33.123456789	0004-06-14 12:37:26.876543211	0004-06-14 12:37:26.876543211	0004-12-30 11:22:33.123456789
 0528-10-27	0528-07-19 12:37:26.876543211	0529-02-03 11:22:33.123456789	0529-02-03 11:22:33.123456789	0528-07-19 12:37:26.876543211	0528-07-19 12:37:26.876543211	0529-02-03 11:22:33.123456789
 1319-02-02	1318-10-25 12:37:26.876543211	1319-05-12 11:22:33.123456789	1319-05-12 11:22:33.123456789	1318-10-25 12:37:26.876543211	1318-10-25 12:37:26.876543211	1319-05-12 11:22:33.123456789
@@ -898,7 +899,7 @@ order by tsval
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-tsval	_c1	_c2	_c3	_c4	_c5	_c6
+tsval	c1	c2	c3	c4	c5	c6
 0004-09-22 18:26:29.519542222	0004-06-15 07:03:56.396085433	0004-12-31 05:49:02.642999011	0004-12-31 05:49:02.642999011	0004-06-15 07:03:56.396085433	0004-06-15 07:03:56.396085433	0004-12-31 05:49:02.642999011
 0528-10-27 08:15:18.941718273	0528-07-19 20:52:45.818261484	0529-02-03 19:37:52.065175062	0529-02-03 19:37:52.065175062	0528-07-19 20:52:45.818261484	0528-07-19 20:52:45.818261484	0529-02-03 19:37:52.065175062
 1319-02-02 16:31:57.778	1318-10-26 05:09:24.654543211	1319-05-13 03:54:30.901456789	1319-05-13 03:54:30.901456789	1318-10-26 05:09:24.654543211	1318-10-26 05:09:24.654543211	1319-05-13 03:54:30.901456789
@@ -1013,7 +1014,7 @@ limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-_c0	_c1
+c0	c1
 109 20:30:40.246913578	89 02:14:26.000000000
 109 20:30:40.246913578	89 02:14:26.000000000
 PREHOOK: query: drop table interval_arithmetic_1


[14/35] hive git commit: HIVE-15207: Implement a capability to detect incorrect sequence numbers (Aihua Xu, reviewed by Chaoyu Tang)

Posted by se...@apache.org.
HIVE-15207: Implement a capability to detect incorrect sequence numbers (Aihua Xu, reviewed by Chaoyu Tang)


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

Branch: refs/heads/hive-14535
Commit: cebd251b95f4d8f1d42a33fe35c59f8f6892875c
Parents: cea954f
Author: Aihua Xu <ai...@apache.org>
Authored: Tue Nov 15 17:22:07 2016 -0500
Committer: Aihua Xu <ai...@apache.org>
Committed: Fri Nov 18 09:10:20 2016 -0500

----------------------------------------------------------------------
 .../org/apache/hive/beeline/HiveSchemaTool.java | 77 +++++++++++++++++++-
 .../org/apache/hive/beeline/TestSchemaTool.java | 34 +++++++++
 2 files changed, 110 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cebd251b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
index cd36ddf..d82c224 100644
--- a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
+++ b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
@@ -28,6 +28,7 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
 import org.apache.commons.io.output.NullOutputStream;
 import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.HiveMetaException;
@@ -38,6 +39,8 @@ import org.apache.hive.beeline.HiveSchemaHelper.NestedScriptParser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.ImmutableMap;
+
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.File;
@@ -51,6 +54,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 public class HiveSchemaTool {
   private String userName = null;
@@ -293,6 +297,74 @@ public class HiveSchemaTool {
     }
   }
 
+  public void doValidate() throws HiveMetaException {
+    System.out.print("Starting metastore validation");
+    validateSequences();
+
+    System.out.print("Done with metastore validation");
+  }
+
+  boolean validateSequences() throws HiveMetaException {
+    Map<String, Pair<String, String>> seqNameToTable =
+        new ImmutableMap.Builder<String, Pair<String, String>>()
+        .put("MDatabase", Pair.of("DBS", "DB_ID"))
+        .put("MRole", Pair.of("ROLES", "ROLE_ID"))
+        .put("MGlobalPrivilege", Pair.of("GLOBAL_PRIVS", "USER_GRANT_ID"))
+        .put("MTable", Pair.of("TBLS","TBL_ID"))
+        .put("MStorageDescriptor", Pair.of("SDS", "SD_ID"))
+        .put("MSerDeInfo", Pair.of("SERDES", "SERDE_ID"))
+        .put("MColumnDescriptor", Pair.of("CDS", "CD_ID"))
+        .put("MTablePrivilege", Pair.of("TBL_PRIVS", "TBL_GRANT_ID"))
+        .put("MTableColumnStatistics", Pair.of("TAB_COL_STATS", "CS_ID"))
+        .put("MPartition", Pair.of("PARTITIONS", "PART_ID"))
+        .put("MPartitionColumnStatistics", Pair.of("PART_COL_STATS", "CS_ID"))
+        .put("MFunction", Pair.of("FUNCS", "FUNC_ID"))
+        .put("MIndex", Pair.of("IDXS", "INDEX_ID"))
+        .put("MStringList", Pair.of("SKEWED_STRING_LIST", "STRING_LIST_ID"))
+        .build();
+
+    System.out.println("Validating sequence number for SEQUENCE_TABLE");
+    Connection conn = getConnectionToMetastore(true);
+    boolean isValid = true;
+    try {
+      Statement stmt = conn.createStatement();
+      for (String seqName : seqNameToTable.keySet()) {
+        String tableName = seqNameToTable.get(seqName).getLeft();
+        String tableKey = seqNameToTable.get(seqName).getRight();
+        String seqQuery = getDbCommandParser(dbType).needsQuotedIdentifier() ?
+            ("select t.\"NEXT_VAL\" from \"SEQUENCE_TABLE\" t WHERE t.\"SEQUENCE_NAME\"='org.apache.hadoop.hive.metastore.model." + seqName + "'")
+            : ("select t.NEXT_VAL from SEQUENCE_TABLE t WHERE t.SEQUENCE_NAME='org.apache.hadoop.hive.metastore.model." + seqName + "'");
+        String maxIdQuery = getDbCommandParser(dbType).needsQuotedIdentifier() ?
+            ("select max(\"" + tableKey + "\") from \"" + tableName + "\"")
+            : ("select max(" + tableKey + ") from " + tableName);
+
+          ResultSet res = stmt.executeQuery(maxIdQuery);
+          if (res.next()) {
+             long maxId = res.getLong(1);
+             if (maxId > 0) {
+               ResultSet resSeq = stmt.executeQuery(seqQuery);
+               if (!resSeq.next() || resSeq.getLong(1) < maxId) {
+                 isValid = false;
+                 System.err.println("Incorrect sequence number: table - " + tableName);
+               }
+             }
+          }
+      }
+
+      return isValid;
+    } catch(SQLException e) {
+        throw new HiveMetaException("Failed to validate sequence number for SEQUENCE_TABLE", e);
+    } finally {
+      if (conn != null) {
+        try {
+          conn.close();
+        } catch (SQLException e) {
+          throw new HiveMetaException("Failed to close metastore connection", e);
+        }
+      }
+    }
+  }
+
   /**
    *  Run pre-upgrade scripts corresponding to a given upgrade script,
    *  if any exist. The errors from pre-upgrade are ignored.
@@ -404,11 +476,12 @@ public class HiveSchemaTool {
                 withDescription("Schema initialization to a version").
                 create("initSchemaTo");
     Option infoOpt = new Option("info", "Show config and schema details");
+    Option validateOpt = new Option("validate", "Validate the database");
 
     OptionGroup optGroup = new OptionGroup();
     optGroup.addOption(upgradeOpt).addOption(initOpt).
                 addOption(help).addOption(upgradeFromOpt).
-                addOption(initToOpt).addOption(infoOpt);
+                addOption(initToOpt).addOption(infoOpt).addOption(validateOpt);
     optGroup.setRequired(true);
 
     Option userNameOpt = OptionBuilder.withArgName("user")
@@ -506,6 +579,8 @@ public class HiveSchemaTool {
       } else if (line.hasOption("initSchemaTo")) {
         schemaVer = line.getOptionValue("initSchemaTo");
         schemaTool.doInit(schemaVer);
+      } else if (line.hasOption("validate")) {
+        schemaTool.doValidate();
       } else {
         System.err.println("no valid option supplied");
         printAndExit(cmdLineOptions);

http://git-wip-us.apache.org/repos/asf/hive/blob/cebd251b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
index 0d5f9c8..8aa4173 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
@@ -70,6 +70,40 @@ public class TestSchemaTool extends TestCase {
   }
 
   /**
+   * Test the sequence validation functionality
+   * @throws Exception
+   */
+  public void testValidateSequences() throws Exception {
+    schemaTool.doInit();
+
+    // Test empty database
+    boolean isValid = (boolean)schemaTool.validateSequences();
+    assertTrue(isValid);
+
+    // Test valid case
+    String[] scripts = new String[] {
+        "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100)",
+        "insert into DBS values(99, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test')"
+    };
+    File scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = schemaTool.validateSequences();
+    assertTrue(isValid);
+
+    // Test invalid case
+    scripts = new String[] {
+        "delete from SEQUENCE_TABLE",
+        "delete from DBS",
+        "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100)",
+        "insert into DBS values(102, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test')"
+    };
+    scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = schemaTool.validateSequences();
+    assertFalse(isValid);
+  }
+
+  /**
    * Test dryrun of schema initialization
    * @throws Exception
    */


[15/35] hive git commit: HIVE-15232: Add notification events for functions and indexes (Mohit Sabharwal, reviewed by Chaoyu Tang)

Posted by se...@apache.org.
HIVE-15232: Add notification events for functions and indexes (Mohit Sabharwal, reviewed by Chaoyu Tang)


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

Branch: refs/heads/hive-14535
Commit: 62d802b871f9654041bb8551b9622b9f5c75e856
Parents: cebd251
Author: Mohit Sabharwal <mo...@cloudera.com>
Authored: Fri Nov 18 10:29:37 2016 -0500
Committer: Mohit Sabharwal <mo...@cloudera.com>
Committed: Fri Nov 18 10:29:37 2016 -0500

----------------------------------------------------------------------
 .../hive/hcatalog/common/HCatConstants.java     |   5 +
 .../listener/DbNotificationListener.java        |  77 ++++-
 .../hcatalog/messaging/AlterIndexMessage.java   |  30 ++
 .../messaging/CreateFunctionMessage.java        |  30 ++
 .../hcatalog/messaging/CreateIndexMessage.java  |  30 ++
 .../hcatalog/messaging/DropFunctionMessage.java |  30 ++
 .../hcatalog/messaging/DropIndexMessage.java    |  30 ++
 .../hcatalog/messaging/HCatEventMessage.java    |  10 +-
 .../hcatalog/messaging/MessageDeserializer.java |  39 ++-
 .../hive/hcatalog/messaging/MessageFactory.java |  38 +++
 .../messaging/json/JSONAlterIndexMessage.java   |  89 ++++++
 .../json/JSONCreateFunctionMessage.java         |  81 ++++++
 .../messaging/json/JSONCreateIndexMessage.java  |  82 ++++++
 .../messaging/json/JSONDropFunctionMessage.java |  81 ++++++
 .../messaging/json/JSONDropIndexMessage.java    |  82 ++++++
 .../messaging/json/JSONMessageDeserializer.java |  56 +++-
 .../messaging/json/JSONMessageFactory.java      |  62 +++-
 .../listener/DummyRawStoreFailEvent.java        |  30 +-
 .../listener/TestDbNotificationListener.java    | 286 ++++++++++++++++++-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  51 ++--
 .../hive/metastore/MetaStoreEventListener.java  |  16 ++
 .../metastore/events/CreateFunctionEvent.java   |  39 +++
 .../metastore/events/DropFunctionEvent.java     |  39 +++
 .../hadoop/hive/metastore/DummyListener.java    |  12 +
 24 files changed, 1282 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
index 72930eb..3998407 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatConstants.java
@@ -162,6 +162,11 @@ public final class HCatConstants {
   public static final String HCAT_DROP_TABLE_EVENT = "DROP_TABLE";
   public static final String HCAT_CREATE_DATABASE_EVENT = "CREATE_DATABASE";
   public static final String HCAT_DROP_DATABASE_EVENT = "DROP_DATABASE";
+  public static final String HCAT_CREATE_FUNCTION_EVENT = "CREATE_FUNCTION";
+  public static final String HCAT_DROP_FUNCTION_EVENT = "DROP_FUNCTION";
+  public static final String HCAT_CREATE_INDEX_EVENT = "CREATE_INDEX";
+  public static final String HCAT_DROP_INDEX_EVENT = "DROP_INDEX";
+  public static final String HCAT_ALTER_INDEX_EVENT = "ALTER_INDEX";
   public static final String HCAT_INSERT_EVENT = "INSERT";
   public static final String HCAT_MESSAGE_VERSION = "HCAT_MESSAGE_VERSION";
   public static final String HCAT_MESSAGE_FORMAT = "HCAT_MESSAGE_FORMAT";

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 0b3d891..ea7520d 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -17,8 +17,13 @@
  */
 package org.apache.hive.hcatalog.listener;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
+import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
+import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreEventListener;
@@ -42,6 +47,8 @@ import org.apache.hadoop.hive.metastore.events.InsertEvent;
 import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
 import org.apache.hive.hcatalog.common.HCatConstants;
 import org.apache.hive.hcatalog.messaging.MessageFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.concurrent.TimeUnit;
 
@@ -221,6 +228,72 @@ public class DbNotificationListener extends MetaStoreEventListener {
     enqueue(event);
   }
 
+  /**
+   * @param fnEvent function event
+   * @throws MetaException
+   */
+  public void onCreateFunction (CreateFunctionEvent fnEvent) throws MetaException {
+    Function fn = fnEvent.getFunction();
+    NotificationEvent event = new NotificationEvent(0, now(),
+        HCatConstants.HCAT_CREATE_FUNCTION_EVENT,
+        msgFactory.buildCreateFunctionMessage(fn).toString());
+    event.setDbName(fn.getDbName());
+    enqueue(event);
+  }
+
+  /**
+   * @param fnEvent function event
+   * @throws MetaException
+   */
+  public void onDropFunction (DropFunctionEvent fnEvent) throws MetaException {
+    Function fn = fnEvent.getFunction();
+    NotificationEvent event = new NotificationEvent(0, now(),
+        HCatConstants.HCAT_DROP_FUNCTION_EVENT,
+        msgFactory.buildDropFunctionMessage(fn).toString());
+    event.setDbName(fn.getDbName());
+    enqueue(event);
+  }
+
+  /**
+   * @param indexEvent index event
+   * @throws MetaException
+   */
+  public void onAddIndex (AddIndexEvent indexEvent) throws MetaException {
+    Index index = indexEvent.getIndex();
+    NotificationEvent event = new NotificationEvent(0, now(),
+        HCatConstants.HCAT_CREATE_INDEX_EVENT,
+        msgFactory.buildCreateIndexMessage(index).toString());
+    event.setDbName(index.getDbName());
+    enqueue(event);
+  }
+
+  /**
+   * @param indexEvent index event
+   * @throws MetaException
+   */
+  public void onDropIndex (DropIndexEvent indexEvent) throws MetaException {
+    Index index = indexEvent.getIndex();
+    NotificationEvent event = new NotificationEvent(0, now(),
+        HCatConstants.HCAT_DROP_INDEX_EVENT,
+        msgFactory.buildDropIndexMessage(index).toString());
+    event.setDbName(index.getDbName());
+    enqueue(event);
+  }
+
+  /**
+   * @param indexEvent index event
+   * @throws MetaException
+   */
+  public void onAlterIndex (AlterIndexEvent indexEvent)  throws MetaException {
+    Index before = indexEvent.getOldIndex();
+    Index after = indexEvent.getNewIndex();
+    NotificationEvent event = new NotificationEvent(0, now(),
+        HCatConstants.HCAT_ALTER_INDEX_EVENT,
+        msgFactory.buildAlterIndexMessage(before, after).toString());
+    event.setDbName(before.getDbName());
+    enqueue(event);
+  }
+
   @Override
   public void onInsert(InsertEvent insertEvent) throws MetaException {
     NotificationEvent event = new NotificationEvent(0, now(), HCatConstants.HCAT_INSERT_EVENT,

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterIndexMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterIndexMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterIndexMessage.java
new file mode 100644
index 0000000..4841dce
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/AlterIndexMessage.java
@@ -0,0 +1,30 @@
+/**
+ * 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.hcatalog.messaging;
+
+/**
+ * HCat message sent when a Index is altered in HCatalog.
+ */
+public abstract class AlterIndexMessage extends HCatEventMessage {
+
+  protected AlterIndexMessage() {
+    super(EventType.ALTER_INDEX);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/CreateFunctionMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/CreateFunctionMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/CreateFunctionMessage.java
new file mode 100644
index 0000000..753c165
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/CreateFunctionMessage.java
@@ -0,0 +1,30 @@
+/**
+ * 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.hcatalog.messaging;
+
+/**
+ * HCat message sent when a Function is created in HCatalog.
+ */
+public abstract class CreateFunctionMessage extends HCatEventMessage {
+
+  protected CreateFunctionMessage() {
+    super(EventType.CREATE_FUNCTION);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/CreateIndexMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/CreateIndexMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/CreateIndexMessage.java
new file mode 100644
index 0000000..192f6de
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/CreateIndexMessage.java
@@ -0,0 +1,30 @@
+/**
+ * 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.hcatalog.messaging;
+
+/**
+ * HCat message sent when a Index is created in HCatalog.
+ */
+public abstract class CreateIndexMessage extends HCatEventMessage {
+
+  protected CreateIndexMessage() {
+    super(EventType.CREATE_INDEX);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/DropFunctionMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/DropFunctionMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/DropFunctionMessage.java
new file mode 100644
index 0000000..19d4d5b
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/DropFunctionMessage.java
@@ -0,0 +1,30 @@
+/**
+ * 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.hcatalog.messaging;
+
+/**
+ * HCat message sent when a Function is dropped in HCatalog.
+ */
+public abstract class DropFunctionMessage extends HCatEventMessage {
+
+  protected DropFunctionMessage() {
+    super(EventType.DROP_FUNCTION);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/DropIndexMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/DropIndexMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/DropIndexMessage.java
new file mode 100644
index 0000000..46b7394
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/DropIndexMessage.java
@@ -0,0 +1,30 @@
+/**
+ * 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.hcatalog.messaging;
+
+/**
+ * HCat message sent when a Index is dropped in HCatalog.
+ */
+public abstract class DropIndexMessage extends HCatEventMessage {
+
+  protected DropIndexMessage() {
+    super(EventType.DROP_INDEX);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/HCatEventMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/HCatEventMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/HCatEventMessage.java
index 538fa68..dca95c7 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/HCatEventMessage.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/HCatEventMessage.java
@@ -40,7 +40,12 @@ public abstract class HCatEventMessage {
     DROP_PARTITION(HCatConstants.HCAT_DROP_PARTITION_EVENT),
     ALTER_TABLE(HCatConstants.HCAT_ALTER_TABLE_EVENT),
     ALTER_PARTITION(HCatConstants.HCAT_ALTER_PARTITION_EVENT),
-    INSERT(HCatConstants.HCAT_INSERT_EVENT);
+    INSERT(HCatConstants.HCAT_INSERT_EVENT),
+    CREATE_FUNCTION(HCatConstants.HCAT_CREATE_FUNCTION_EVENT),
+    DROP_FUNCTION(HCatConstants.HCAT_DROP_FUNCTION_EVENT),
+    CREATE_INDEX(HCatConstants.HCAT_CREATE_INDEX_EVENT),
+    DROP_INDEX(HCatConstants.HCAT_DROP_INDEX_EVENT),
+    ALTER_INDEX(HCatConstants.HCAT_ALTER_INDEX_EVENT);
 
     private String typeString;
 
@@ -85,7 +90,7 @@ public abstract class HCatEventMessage {
    * Getter for the timestamp associated with the operation.
    * @return Timestamp (Long - seconds since epoch).
    */
-  public abstract Long   getTimestamp();
+  public abstract Long getTimestamp();
 
   /**
    * Class invariant. Checked after construction or deserialization.
@@ -97,7 +102,6 @@ public abstract class HCatEventMessage {
       throw new IllegalStateException("Event-type unset.");
     if (getDB() == null)
       throw new IllegalArgumentException("DB-name unset.");
-
     return this;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.java
index 8ea3998..e18780f 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageDeserializer.java
@@ -19,6 +19,9 @@
 
 package org.apache.hive.hcatalog.messaging;
 
+import org.apache.hive.hcatalog.messaging.json.JSONCreateFunctionMessage;
+import org.apache.hive.hcatalog.messaging.json.JSONDropFunctionMessage;
+
 /**
  * Interface for converting HCat events from String-form back to HCatEventMessage instances.
  */
@@ -46,9 +49,18 @@ public abstract class MessageDeserializer {
       return getAlterPartitionMessage(messageBody);
     case DROP_PARTITION:
       return getDropPartitionMessage(messageBody);
+    case CREATE_FUNCTION:
+      return getCreateFunctionMessage(messageBody);
+    case DROP_FUNCTION:
+      return getDropFunctionMessage(messageBody);
+    case CREATE_INDEX:
+      return getCreateIndexMessage(messageBody);
+    case DROP_INDEX:
+      return getDropIndexMessage(messageBody);
+    case ALTER_INDEX:
+      return getAlterIndexMessage(messageBody);
     case INSERT:
       return getInsertMessage(messageBody);
-
     default:
       throw new IllegalArgumentException("Unsupported event-type: " + eventTypeString);
     }
@@ -99,6 +111,31 @@ public abstract class MessageDeserializer {
   public abstract DropPartitionMessage getDropPartitionMessage(String messageBody);
 
   /**
+   * Method to de-serialize CreateFunctionMessage instance.
+   */
+  public abstract CreateFunctionMessage getCreateFunctionMessage(String messageBody);
+
+  /**
+   * Method to de-serialize DropFunctionMessage instance.
+   */
+  public abstract DropFunctionMessage getDropFunctionMessage(String messageBody);
+
+  /**
+   * Method to de-serialize CreateIndexMessage instance.
+   */
+  public abstract CreateIndexMessage getCreateIndexMessage(String messageBody);
+
+  /**
+   * Method to de-serialize DropIndexMessage instance.
+   */
+  public abstract DropIndexMessage getDropIndexMessage(String messageBody);
+
+  /**
+   * Method to de-serialize AlterIndexMessage instance.
+   */
+  public abstract AlterIndexMessage getAlterIndexMessage(String messageBody);
+
+  /**
    * Method to deserialize InsertMessage
    * @param messageBody the message in serialized form
    * @return message in object form

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
index 0710dd0..44574fe 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/MessageFactory.java
@@ -22,6 +22,8 @@ package org.apache.hive.hcatalog.messaging;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -163,6 +165,42 @@ public abstract class MessageFactory {
   public abstract DropPartitionMessage buildDropPartitionMessage(Table table, Iterator<Partition> partitions);
 
   /**
+   * Factory method for CreateFunctionMessage.
+   * @param fn The Function being added.
+   * @return CreateFunctionMessage instance.
+   */
+  public abstract CreateFunctionMessage buildCreateFunctionMessage(Function fn);
+
+  /**
+   * Factory method for DropFunctionMessage.
+   * @param fn The Function being dropped.
+   * @return DropFunctionMessage instance.
+   */
+  public abstract DropFunctionMessage buildDropFunctionMessage(Function fn);
+
+  /**
+   * Factory method for CreateIndexMessage.
+   * @param idx The Index being added.
+   * @return CreateIndexMessage instance.
+   */
+  public abstract CreateIndexMessage buildCreateIndexMessage(Index idx);
+
+  /**
+   * Factory method for DropIndexMessage.
+   * @param idx The Index being dropped.
+   * @return DropIndexMessage instance.
+   */
+  public abstract DropIndexMessage buildDropIndexMessage(Index idx);
+
+  /**
+   * Factory method for AlterIndexMessage.
+   * @param before The index before the alter
+   * @param after The index after the alter
+   * @return AlterIndexMessage
+   */
+  public abstract AlterIndexMessage buildAlterIndexMessage(Index before, Index after);
+
+  /**
    * Factory method for building insert message
    * @param db Name of the database the insert occurred in
    * @param table Name of the table the insert occurred in

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterIndexMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterIndexMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterIndexMessage.java
new file mode 100644
index 0000000..25b0987
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONAlterIndexMessage.java
@@ -0,0 +1,89 @@
+/**
+ * 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.hcatalog.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hive.hcatalog.messaging.AlterIndexMessage;
+import org.apache.thrift.TException;
+
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of AlterIndexMessage.
+ */
+public class JSONAlterIndexMessage extends AlterIndexMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, beforeIndexObjJson, afterIndexObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONAlterIndexMessage() {}
+
+  public JSONAlterIndexMessage(String server, String servicePrincipal, Index before, Index after,
+                               Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = after.getDbName();
+    this.timestamp = timestamp;
+    try {
+      this.beforeIndexObjJson = JSONMessageFactory.createIndexObjJson(before);
+      this.afterIndexObjJson = JSONMessageFactory.createIndexObjJson(after);
+    } catch (TException ex) {
+      throw new IllegalArgumentException("Could not serialize Index object", ex);
+    }
+
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  public String getBeforeIndexObjJson() {
+    return beforeIndexObjJson;
+  }
+
+  public String getAfterIndexObjJson() {
+    return afterIndexObjJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONCreateFunctionMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONCreateFunctionMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONCreateFunctionMessage.java
new file mode 100644
index 0000000..fb883fc
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONCreateFunctionMessage.java
@@ -0,0 +1,81 @@
+/**
+ * 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.hcatalog.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hive.hcatalog.messaging.CreateFunctionMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of CreateFunctionMessage.
+ */
+public class JSONCreateFunctionMessage extends CreateFunctionMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, functionObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONCreateFunctionMessage() {}
+
+  public JSONCreateFunctionMessage(String server, String servicePrincipal, Function fn, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = fn.getDbName();
+    this.timestamp = timestamp;
+    try {
+      this.functionObjJson = JSONMessageFactory.createFunctionObjJson(fn);
+    } catch (TException ex) {
+      throw new IllegalArgumentException("Could not serialize Function object", ex);
+    }
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  public String getFunctionObjJson() {
+    return functionObjJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONCreateIndexMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONCreateIndexMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONCreateIndexMessage.java
new file mode 100644
index 0000000..8d83149
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONCreateIndexMessage.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hive.hcatalog.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hive.hcatalog.messaging.CreateIndexMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of CreateIndexMessage.
+ */
+public class JSONCreateIndexMessage extends CreateIndexMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, indexObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONCreateIndexMessage() {}
+
+  public JSONCreateIndexMessage(String server, String servicePrincipal, Index index, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = index.getDbName();
+    try {
+      this.indexObjJson = JSONMessageFactory.createIndexObjJson(index);
+    } catch (TException ex) {
+      throw new IllegalArgumentException("Could not serialize Index object", ex);
+    }
+
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  public String getIndexObjJson() {
+    return indexObjJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONDropFunctionMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONDropFunctionMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONDropFunctionMessage.java
new file mode 100644
index 0000000..334e36f
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONDropFunctionMessage.java
@@ -0,0 +1,81 @@
+/**
+ * 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.hcatalog.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hive.hcatalog.messaging.DropFunctionMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of CreateDatabaseMessage.
+ */
+public class JSONDropFunctionMessage extends DropFunctionMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, functionObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONDropFunctionMessage() {}
+
+  public JSONDropFunctionMessage(String server, String servicePrincipal, Function fn, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = fn.getDbName();
+    this.timestamp = timestamp;
+    try {
+      this.functionObjJson = JSONMessageFactory.createFunctionObjJson(fn);
+    } catch (TException ex) {
+      throw new IllegalArgumentException("Could not serialize Function object", ex);
+    }
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  public String getFunctionObjJson() {
+    return functionObjJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONDropIndexMessage.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONDropIndexMessage.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONDropIndexMessage.java
new file mode 100644
index 0000000..bacaa1d
--- /dev/null
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONDropIndexMessage.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hive.hcatalog.messaging.json;
+
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hive.hcatalog.messaging.DropIndexMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+
+/**
+ * JSON Implementation of DropIndexMessage.
+ */
+public class JSONDropIndexMessage extends DropIndexMessage {
+
+  @JsonProperty
+  String server, servicePrincipal, db, indexObjJson;
+
+  @JsonProperty
+  Long timestamp;
+
+  /**
+   * Default constructor, required for Jackson.
+   */
+  public JSONDropIndexMessage() {}
+
+  public JSONDropIndexMessage(String server, String servicePrincipal, Index index, Long timestamp) {
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.db = index.getDbName();
+    try {
+      this.indexObjJson = JSONMessageFactory.createIndexObjJson(index);
+    } catch (TException ex) {
+      throw new IllegalArgumentException("Could not serialize Index object", ex);
+    }
+
+    this.timestamp = timestamp;
+    checkValid();
+  }
+
+  @Override
+  public String getDB() { return db; }
+
+  @Override
+  public String getServer() { return server; }
+
+  @Override
+  public String getServicePrincipal() { return servicePrincipal; }
+
+  @Override
+  public Long getTimestamp() { return timestamp; }
+
+  public String getIndexObjJson() {
+    return indexObjJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageDeserializer.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageDeserializer.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageDeserializer.java
index 834fdde..bd45d09 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageDeserializer.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageDeserializer.java
@@ -19,13 +19,17 @@
 
 package org.apache.hive.hcatalog.messaging.json;
 
-import org.apache.hive.hcatalog.common.HCatConstants;
 import org.apache.hive.hcatalog.messaging.AddPartitionMessage;
+import org.apache.hive.hcatalog.messaging.AlterIndexMessage;
 import org.apache.hive.hcatalog.messaging.AlterPartitionMessage;
 import org.apache.hive.hcatalog.messaging.AlterTableMessage;
 import org.apache.hive.hcatalog.messaging.CreateDatabaseMessage;
+import org.apache.hive.hcatalog.messaging.CreateFunctionMessage;
+import org.apache.hive.hcatalog.messaging.CreateIndexMessage;
 import org.apache.hive.hcatalog.messaging.CreateTableMessage;
 import org.apache.hive.hcatalog.messaging.DropDatabaseMessage;
+import org.apache.hive.hcatalog.messaging.DropFunctionMessage;
+import org.apache.hive.hcatalog.messaging.DropIndexMessage;
 import org.apache.hive.hcatalog.messaging.DropPartitionMessage;
 import org.apache.hive.hcatalog.messaging.DropTableMessage;
 import org.apache.hive.hcatalog.messaging.InsertMessage;
@@ -125,6 +129,56 @@ public class JSONMessageDeserializer extends MessageDeserializer {
   }
 
   @Override
+  public CreateFunctionMessage getCreateFunctionMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONCreateFunctionMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONCreateFunctionMessage.", exception);
+    }
+  }
+
+  @Override
+  public DropFunctionMessage getDropFunctionMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONDropFunctionMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONDropDatabaseMessage.", exception);
+    }
+  }
+
+  @Override
+  public CreateIndexMessage getCreateIndexMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONCreateIndexMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONCreateIndexMessage.", exception);
+    }
+  }
+
+  @Override
+  public DropIndexMessage getDropIndexMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONDropIndexMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONDropIndexMessage.", exception);
+    }
+  }
+
+  @Override
+  public AlterIndexMessage getAlterIndexMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAlterIndexMessage.class);
+    }
+    catch (Exception exception) {
+      throw new IllegalArgumentException("Could not construct JSONAlterIndexMessage.", exception);
+    }
+  }
+
+  @Override
   public InsertMessage getInsertMessage(String messageBody) {
     try {
       return mapper.readValue(messageBody, JSONInsertMessage.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java
index 6b74b54..251084f 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/messaging/json/JSONMessageFactory.java
@@ -19,12 +19,17 @@
 
 package org.apache.hive.hcatalog.messaging.json;
 
-import com.google.common.base.Function;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hive.hcatalog.messaging.AlterIndexMessage;
+import org.apache.hive.hcatalog.messaging.CreateFunctionMessage;
+import org.apache.hive.hcatalog.messaging.CreateIndexMessage;
+import org.apache.hive.hcatalog.messaging.DropFunctionMessage;
+import org.apache.hive.hcatalog.messaging.DropIndexMessage;
 import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hive.hcatalog.messaging.AddPartitionMessage;
@@ -38,9 +43,13 @@ import org.apache.hive.hcatalog.messaging.DropTableMessage;
 import org.apache.hive.hcatalog.messaging.InsertMessage;
 import org.apache.hive.hcatalog.messaging.MessageDeserializer;
 import org.apache.hive.hcatalog.messaging.MessageFactory;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+import org.apache.thrift.protocol.TJSONProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
-import java.util.Arrays;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -54,7 +63,6 @@ public class JSONMessageFactory extends MessageFactory {
 
   private static final Logger LOG = LoggerFactory.getLogger(JSONMessageFactory.class.getName());
 
-
   private static JSONMessageDeserializer deserializer = new JSONMessageDeserializer();
 
   @Override
@@ -121,6 +129,36 @@ public class JSONMessageFactory extends MessageFactory {
   }
 
   @Override
+  public CreateFunctionMessage buildCreateFunctionMessage(Function fn) {
+    return new JSONCreateFunctionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, fn,
+        now());
+  }
+
+  @Override
+  public DropFunctionMessage buildDropFunctionMessage(Function fn) {
+    return new JSONDropFunctionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, fn,
+        now());
+  }
+
+  @Override
+  public CreateIndexMessage buildCreateIndexMessage(Index idx) {
+    return new JSONCreateIndexMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, idx,
+        now());
+  }
+
+  @Override
+  public DropIndexMessage buildDropIndexMessage(Index idx) {
+    return new JSONDropIndexMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, idx,
+        now());
+  }
+
+  @Override
+  public AlterIndexMessage buildAlterIndexMessage(Index before, Index after) {
+    return new JSONAlterIndexMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, before, after,
+        now());
+  }
+
+  @Override
   public InsertMessage buildInsertMessage(String db, String table, Map<String,String> partKeyVals,
                                           List<String> files) {
     return new JSONInsertMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, db, table, partKeyVals,
@@ -140,11 +178,21 @@ public class JSONMessageFactory extends MessageFactory {
   }
 
   private static List<Map<String, String>> getPartitionKeyValues(final Table table, Iterator<Partition> iterator) {
-    return Lists.newArrayList(Iterators.transform(iterator, new Function<Partition, Map<String, String>>() {
+    return Lists.newArrayList(Iterators.transform(iterator, new com.google.common.base.Function<Partition, Map<String, String>>() {
       @Override
       public Map<String, String> apply(@Nullable Partition partition) {
         return getPartitionKeyValues(table, partition);
       }
     }));
   }
-}
+
+  static String createFunctionObjJson(Function functionObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(functionObj, "UTF-8");
+  }
+
+  static String createIndexObjJson(Index indexObj) throws TException {
+    TSerializer serializer = new TSerializer(new TJSONProtocol.Factory());
+    return serializer.toString(indexObj, "UTF-8");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 4a7801b..5282a5a 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -300,7 +300,11 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
 
   @Override
   public boolean addIndex(Index index) throws InvalidObjectException, MetaException {
-    return objectStore.addIndex(index);
+    if (shouldEventSucceed) {
+      return objectStore.addIndex(index);
+    } else {
+      throw new RuntimeException("Event failed.");
+    }
   }
 
   @Override
@@ -312,7 +316,11 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   @Override
   public boolean dropIndex(String dbName, String origTableName, String indexName)
       throws MetaException {
-    return objectStore.dropIndex(dbName, origTableName, indexName);
+    if (shouldEventSucceed) {
+      return objectStore.dropIndex(dbName, origTableName, indexName);
+    } else {
+      throw new RuntimeException("Event failed.");
+    }
   }
 
   @Override
@@ -330,7 +338,11 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   @Override
   public void alterIndex(String dbName, String baseTblName, String name, Index newIndex)
       throws InvalidObjectException, MetaException {
-    objectStore.alterIndex(dbName, baseTblName, name, newIndex);
+    if (shouldEventSucceed) {
+      objectStore.alterIndex(dbName, baseTblName, name, newIndex);
+    } else {
+      throw new RuntimeException("Event failed.");
+    }
   }
 
   @Override
@@ -751,7 +763,11 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   @Override
   public void createFunction(Function func) throws InvalidObjectException,
       MetaException {
-    objectStore.createFunction(func);
+    if (shouldEventSucceed) {
+      objectStore.createFunction(func);
+    } else {
+      throw new RuntimeException("Event failed.");
+    }
   }
 
   @Override
@@ -764,7 +780,11 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   public void dropFunction(String dbName, String funcName)
       throws MetaException, NoSuchObjectException, InvalidObjectException,
       InvalidInputException {
-    objectStore.dropFunction(dbName, funcName);
+    if (shouldEventSucceed) {
+      objectStore.dropFunction(dbName, funcName);
+    } else {
+      throw new RuntimeException("Event failed.");
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index 1cd32d5..4f97cf4 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -23,8 +23,20 @@ import static junit.framework.Assert.assertNull;
 import static junit.framework.Assert.assertTrue;
 import static junit.framework.Assert.fail;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.FunctionType;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.ResourceType;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.htrace.fasterxml.jackson.core.JsonFactory;
+import org.apache.htrace.fasterxml.jackson.core.JsonParser;
+import org.apache.htrace.fasterxml.jackson.databind.JsonNode;
+import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.htrace.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.protocol.TJSONProtocol;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
@@ -46,6 +58,8 @@ import org.apache.hive.hcatalog.common.HCatConstants;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.lang.reflect.Field;
 import java.util.ArrayList;
@@ -55,7 +69,6 @@ import java.util.List;
 import java.util.Map;
 
 public class TestDbNotificationListener {
-
   private static final Logger LOG = LoggerFactory.getLogger(TestDbNotificationListener.class.getName());
   private static final int EVENTS_TTL = 30;
   private static final int CLEANUP_SLEEP_TIME = 10;
@@ -402,7 +415,7 @@ public class TestDbNotificationListener {
 
     partition = new Partition(Arrays.asList("tomorrow"), "default", "dropPartTable",
         startTime, startTime, sd, emptyParameters);
-    msClient.add_partition(partition);
+      msClient.add_partition(partition);
     DummyRawStoreFailEvent.setEventSucceed(false);
     try {
       msClient.dropPartition("default", "dropparttable", Arrays.asList("tomorrow"), false);
@@ -415,6 +428,244 @@ public class TestDbNotificationListener {
   }
 
   @Test
+  public void createFunction() throws Exception {
+    String funcName = "createFunction";
+    String dbName = "default";
+    String ownerName = "me";
+    String funcClass = "o.a.h.h.myfunc";
+    String funcResource = "file:/tmp/somewhere";
+    Function func = new Function(funcName, dbName, funcClass, ownerName, PrincipalType.USER,
+        startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR,
+        funcResource)));
+    msClient.createFunction(func);
+    NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(1, rsp.getEventsSize());
+    NotificationEvent event = rsp.getEvents().get(0);
+    assertEquals(firstEventId + 1, event.getEventId());
+    assertTrue(event.getEventTime() >= startTime);
+    assertEquals(HCatConstants.HCAT_CREATE_FUNCTION_EVENT, event.getEventType());
+    assertEquals(dbName, event.getDbName());
+    Function funcObj = getFunctionObj(getJsonTree(event));
+    assertEquals(dbName, funcObj.getDbName());
+    assertEquals(funcName, funcObj.getFunctionName());
+    assertEquals(funcClass, funcObj.getClassName());
+    assertEquals(ownerName, funcObj.getOwnerName());
+    assertEquals(FunctionType.JAVA, funcObj.getFunctionType());
+    assertEquals(1, funcObj.getResourceUrisSize());
+    assertEquals(ResourceType.JAR, funcObj.getResourceUris().get(0).getResourceType());
+    assertEquals(funcResource, funcObj.getResourceUris().get(0).getUri());
+
+    DummyRawStoreFailEvent.setEventSucceed(false);
+    func = new Function("createFunction2", dbName, "o.a.h.h.myfunc2", "me", PrincipalType.USER,
+        startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR,
+        "file:/tmp/somewhere2")));
+    try {
+      msClient.createFunction(func);
+    } catch (Exception ex) {
+      // expected
+    }
+
+    rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(1, rsp.getEventsSize());
+  }
+
+  @Test
+  public void dropFunction() throws Exception {
+    String funcName = "dropfunctiontest";
+    String dbName = "default";
+    String ownerName = "me";
+    String funcClass = "o.a.h.h.dropFunctionTest";
+    String funcResource = "file:/tmp/somewhere";
+    Function func = new Function(funcName, dbName, funcClass, ownerName, PrincipalType.USER,
+        startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR,
+        funcResource)));
+    msClient.createFunction(func);
+    msClient.dropFunction(dbName, funcName);
+    NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(2, rsp.getEventsSize());
+    NotificationEvent event = rsp.getEvents().get(1);
+    assertEquals(firstEventId + 2, event.getEventId());
+    assertTrue(event.getEventTime() >= startTime);
+    assertEquals(HCatConstants.HCAT_DROP_FUNCTION_EVENT, event.getEventType());
+    assertEquals(dbName, event.getDbName());
+    Function funcObj = getFunctionObj(getJsonTree(event));
+    assertEquals(dbName, funcObj.getDbName());
+    assertEquals(funcName, funcObj.getFunctionName());
+    assertEquals(funcClass, funcObj.getClassName());
+    assertEquals(ownerName, funcObj.getOwnerName());
+    assertEquals(FunctionType.JAVA, funcObj.getFunctionType());
+    assertEquals(1, funcObj.getResourceUrisSize());
+    assertEquals(ResourceType.JAR, funcObj.getResourceUris().get(0).getResourceType());
+    assertEquals(funcResource, funcObj.getResourceUris().get(0).getUri());
+
+    func = new Function("dropfunctiontest2", dbName, "o.a.h.h.dropFunctionTest2", "me",
+        PrincipalType.USER,  startTime, FunctionType.JAVA, Arrays.asList(
+        new ResourceUri(ResourceType.JAR, "file:/tmp/somewhere2")));
+    msClient.createFunction(func);
+    DummyRawStoreFailEvent.setEventSucceed(false);
+    try {
+      msClient.dropFunction(dbName, "dropfunctiontest2");
+    } catch (Exception ex) {
+      // expected
+    }
+
+    rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(3, rsp.getEventsSize());
+  }
+
+  @Test
+  public void createIndex() throws Exception {
+    String indexName = "createIndex";
+    String dbName = "default";
+    String tableName = "createIndexTable";
+    String indexTableName = tableName + "__" + indexName + "__";
+    int startTime = (int)(System.currentTimeMillis() / 1000);
+    List<FieldSchema> cols = new ArrayList<FieldSchema>();
+    cols.add(new FieldSchema("col1", "int", ""));
+    SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+    Map<String, String> params = new HashMap<String, String>();
+    params.put("key", "value");
+    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
+        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+    Table table = new Table(tableName, dbName, "me", startTime, startTime, 0, sd, null,
+        emptyParameters, null, null, null);
+    msClient.createTable(table);
+    Index index = new Index(indexName, null, "default", tableName, startTime, startTime,
+        indexTableName, sd, emptyParameters, false);
+    Table indexTable = new Table(indexTableName, dbName, "me", startTime, startTime, 0, sd, null,
+        emptyParameters, null, null, null);
+    msClient.createIndex(index, indexTable);
+    NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(3, rsp.getEventsSize());
+    NotificationEvent event = rsp.getEvents().get(2);
+    assertEquals(firstEventId + 3, event.getEventId());
+    assertTrue(event.getEventTime() >= startTime);
+    assertEquals(HCatConstants.HCAT_CREATE_INDEX_EVENT, event.getEventType());
+    assertEquals(dbName, event.getDbName());
+    Index indexObj = getIndexObj(getJsonTree(event));
+    assertEquals(dbName, indexObj.getDbName());
+    assertEquals(indexName, indexObj.getIndexName());
+    assertEquals(tableName, indexObj.getOrigTableName());
+    assertEquals(indexTableName, indexObj.getIndexTableName());
+
+    DummyRawStoreFailEvent.setEventSucceed(false);
+    index = new Index("createIndexTable2", null, "default", tableName, startTime, startTime,
+        "createIndexTable2__createIndexTable2__", sd, emptyParameters, false);
+    Table indexTable2 = new Table("createIndexTable2__createIndexTable2__", dbName, "me",
+        startTime, startTime, 0, sd, null, emptyParameters, null, null, null);
+    try {
+      msClient.createIndex(index, indexTable2);
+    } catch (Exception ex) {
+      // expected
+    }
+
+    rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(3, rsp.getEventsSize());
+  }
+
+  @Test
+  public void dropIndex() throws Exception {
+    String indexName = "dropIndex";
+    String dbName = "default";
+    String tableName = "dropIndexTable";
+    String indexTableName = tableName + "__" + indexName + "__";
+    int startTime = (int)(System.currentTimeMillis() / 1000);
+    List<FieldSchema> cols = new ArrayList<FieldSchema>();
+    cols.add(new FieldSchema("col1", "int", ""));
+    SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+    Map<String, String> params = new HashMap<String, String>();
+    params.put("key", "value");
+    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
+        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+    Table table = new Table(tableName, dbName, "me", startTime, startTime, 0, sd, null,
+        emptyParameters, null, null, null);
+    msClient.createTable(table);
+    Index index = new Index(indexName, null, "default", tableName, startTime, startTime,
+        indexTableName, sd, emptyParameters, false);
+    Table indexTable = new Table(indexTableName, dbName, "me", startTime, startTime, 0, sd, null,
+        emptyParameters, null, null, null);
+    msClient.createIndex(index, indexTable);
+    msClient.dropIndex(dbName, tableName, indexName, true); // drops index and indexTable
+    NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(4, rsp.getEventsSize());
+    NotificationEvent event = rsp.getEvents().get(3);
+    assertEquals(firstEventId + 4, event.getEventId());
+    assertTrue(event.getEventTime() >= startTime);
+    assertEquals(HCatConstants.HCAT_DROP_INDEX_EVENT, event.getEventType());
+    assertEquals(dbName, event.getDbName());
+    Index indexObj = getIndexObj(getJsonTree(event));
+    assertEquals(dbName, indexObj.getDbName());
+    assertEquals(indexName.toLowerCase(), indexObj.getIndexName());
+    assertEquals(tableName.toLowerCase(), indexObj.getOrigTableName());
+    assertEquals(indexTableName.toLowerCase(), indexObj.getIndexTableName());
+
+    index = new Index("dropIndexTable2", null, "default", tableName, startTime, startTime,
+        "dropIndexTable__dropIndexTable2__", sd, emptyParameters, false);
+    Table indexTable2 = new Table("dropIndexTable__dropIndexTable2__", dbName, "me", startTime,
+        startTime, 0, sd, null, emptyParameters, null, null, null);
+    msClient.createIndex(index, indexTable2);
+    DummyRawStoreFailEvent.setEventSucceed(false);
+    try {
+      msClient.dropIndex(dbName, tableName, "dropIndex2", true); // drops index and indexTable
+    } catch (Exception ex) {
+      // expected
+    }
+
+    rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(6, rsp.getEventsSize());
+  }
+
+  @Test
+  public void alterIndex() throws Exception {
+    String indexName = "alterIndex";
+    String dbName = "default";
+    String tableName = "alterIndexTable";
+    String indexTableName = tableName + "__" + indexName + "__";
+    int startTime = (int)(System.currentTimeMillis() / 1000);
+    List<FieldSchema> cols = new ArrayList<FieldSchema>();
+    cols.add(new FieldSchema("col1", "int", ""));
+    SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
+    Map<String, String> params = new HashMap<String, String>();
+    params.put("key", "value");
+    StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
+        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+    Table table = new Table(tableName, dbName, "me", startTime, startTime, 0, sd, null,
+        emptyParameters, null, null, null);
+    msClient.createTable(table);
+    Index oldIndex = new Index(indexName, null, "default", tableName, startTime, startTime,
+        indexTableName, sd, emptyParameters, false);
+    Table oldIndexTable = new Table(indexTableName, dbName, "me", startTime, startTime, 0, sd, null,
+        emptyParameters, null, null, null);
+    msClient.createIndex(oldIndex, oldIndexTable); // creates index and index table
+    Index newIndex = new Index(indexName, null, "default", tableName, startTime, startTime + 1,
+        indexTableName, sd, emptyParameters, false);
+    msClient.alter_index(dbName, tableName, indexName, newIndex);
+    NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(4, rsp.getEventsSize());
+    NotificationEvent event = rsp.getEvents().get(3);
+    assertEquals(firstEventId + 4, event.getEventId());
+    assertTrue(event.getEventTime() >= startTime);
+    assertEquals(HCatConstants.HCAT_ALTER_INDEX_EVENT, event.getEventType());
+    assertEquals(dbName, event.getDbName());
+    Index indexObj = getIndexObj(getJsonTree(event), "afterIndexObjJson");
+    assertEquals(dbName, indexObj.getDbName());
+    assertEquals(indexName, indexObj.getIndexName());
+    assertEquals(tableName, indexObj.getOrigTableName());
+    assertEquals(indexTableName, indexObj.getIndexTableName());
+    assertTrue(indexObj.getCreateTime() < indexObj.getLastAccessTime());
+
+    DummyRawStoreFailEvent.setEventSucceed(false);
+    try {
+      msClient.alter_index(dbName, tableName, indexName, newIndex);
+    } catch (Exception ex) {
+      // expected
+    }
+
+    rsp = msClient.getNextNotification(firstEventId, 0, null);
+    assertEquals(4, rsp.getEventsSize());
+  }
+
+  @Test
   public void insertTable() throws Exception {
     List<FieldSchema> cols = new ArrayList<FieldSchema>();
     cols.add(new FieldSchema("col1", "int", "nocomment"));
@@ -700,7 +951,6 @@ public class TestDbNotificationListener {
     assertEquals(firstEventId + 24, event.getEventId());
     assertEquals(HCatConstants.HCAT_ALTER_PARTITION_EVENT, event.getEventType());
     assertTrue(event.getMessage().matches(".*\"ds\":\"todaytwo\".*"));
-
    }
 
   @Test
@@ -721,4 +971,30 @@ public class TestDbNotificationListener {
     LOG.info("second trigger done");
     assertEquals(0, rsp2.getEventsSize());
   }
+
+  private ObjectNode getJsonTree(NotificationEvent event) throws Exception {
+    JsonParser jsonParser = (new JsonFactory()).createJsonParser(event.getMessage());
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(jsonParser, ObjectNode.class);
+  }
+
+  private Function getFunctionObj(JsonNode jsonTree) throws Exception {
+    TDeserializer deSerializer = new TDeserializer(new TJSONProtocol.Factory());
+    Function funcObj = new Function();
+    String tableJson = jsonTree.get("functionObjJson").asText();
+    deSerializer.deserialize(funcObj, tableJson, "UTF-8");
+    return funcObj;
+  }
+
+  private Index getIndexObj(JsonNode jsonTree) throws Exception {
+    return getIndexObj(jsonTree, "indexObjJson");
+  }
+
+  private Index getIndexObj(JsonNode jsonTree, String indexObjKey) throws Exception {
+    TDeserializer deSerializer = new TDeserializer(new TJSONProtocol.Factory());
+    Index indexObj = new Index();
+    String tableJson = jsonTree.get(indexObjKey).asText();
+    deSerializer.deserialize(indexObj, tableJson, "UTF-8");
+    return indexObj;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/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 c0ef25e..48bebb2 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -176,8 +176,10 @@ import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
@@ -3909,7 +3911,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           + " idx=" + index_name + " newidx=" + newIndex.getIndexName());
       newIndex.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
           .currentTimeMillis() / 1000));
-
       boolean success = false;
       Exception ex = null;
       Index oldIndex = null;
@@ -3917,9 +3918,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       try {
         ms.openTransaction();
         oldIndex = get_index_by_name(dbname, base_table_name, index_name);
-
         firePreEvent(new PreAlterIndexEvent(oldIndex, newIndex, this));
-
         ms.alterIndex(dbname, base_table_name, index_name, newIndex);
         if (transactionalListeners.size() > 0) {
           AlterIndexEvent alterIndexEvent = new AlterIndexEvent(oldIndex, newIndex, true, this);
@@ -4585,16 +4584,12 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     private Index add_index_core(final RawStore ms, final Index index, final Table indexTable)
         throws InvalidObjectException, AlreadyExistsException, MetaException {
-
       boolean success = false, indexTableCreated = false;
-
       String[] qualified =
           MetaStoreUtils.getQualifiedName(index.getDbName(), index.getIndexTableName());
-
       try {
         ms.openTransaction();
         firePreEvent(new PreAddIndexEvent(index, this));
-
         Index old_index = null;
         try {
           old_index = get_index_by_name(index.getDbName(), index
@@ -4628,7 +4623,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
         index.setCreateTime((int) time);
         index.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(time));
-
         if (ms.addIndex(index)) {
           if (transactionalListeners.size() > 0) {
             AddIndexEvent addIndexEvent = new AddIndexEvent(index, true, this);
@@ -4687,21 +4681,16 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         final String dbName, final String tblName,
         final String indexName, final boolean deleteData) throws NoSuchObjectException,
         MetaException, TException, IOException, InvalidObjectException, InvalidInputException {
-
       boolean success = false;
       Index index = null;
       Path tblPath = null;
       List<Path> partPaths = null;
       try {
         ms.openTransaction();
-
         // drop the underlying index table
         index = get_index_by_name(dbName, tblName, indexName);  // throws exception if not exists
-
         firePreEvent(new PreDropIndexEvent(index, this));
-
         ms.dropIndex(dbName, tblName, indexName);
-
         String idxTblName = index.getIndexTableName();
         if (idxTblName != null) {
           String[] qualified = MetaStoreUtils.getQualifiedName(index.getDbName(), idxTblName);
@@ -4722,7 +4711,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           // Drop the partitions and get a list of partition locations which need to be deleted
           partPaths = dropPartitionsAndGetLocations(ms, qualified[0], qualified[1], tblPath,
               tbl.getPartitionKeys(), deleteData);
-
           if (!ms.dropTable(qualified[0], qualified[1])) {
             throw new MetaException("Unable to drop underlying data table "
                 + qualified[0] + "." + qualified[1] + " for index " + indexName);
@@ -6181,31 +6169,43 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         InvalidObjectException, MetaException, NoSuchObjectException,
         TException {
       validateFunctionInfo(func);
-
       boolean success = false;
       RawStore ms = getMS();
       try {
         ms.openTransaction();
-
         Database db = ms.getDatabase(func.getDbName());
         if (db == null) {
           throw new NoSuchObjectException("The database " + func.getDbName() + " does not exist");
         }
+
         Function existingFunc = ms.getFunction(func.getDbName(), func.getFunctionName());
         if (existingFunc != null) {
           throw new AlreadyExistsException(
               "Function " + func.getFunctionName() + " already exists");
         }
 
-        // set create time
         long time = System.currentTimeMillis() / 1000;
         func.setCreateTime((int) time);
         ms.createFunction(func);
+        if (transactionalListeners.size() > 0) {
+          CreateFunctionEvent createFunctionEvent = new CreateFunctionEvent(func, true, this);
+          for (MetaStoreEventListener transactionalListener : transactionalListeners) {
+            transactionalListener.onCreateFunction(createFunctionEvent);
+          }
+        }
+
         success = ms.commitTransaction();
       } finally {
         if (!success) {
           ms.rollbackTransaction();
         }
+
+        if (listeners.size() > 0) {
+          CreateFunctionEvent createFunctionEvent = new CreateFunctionEvent(func, success, this);
+          for (MetaStoreEventListener listener : listeners) {
+            listener.onCreateFunction(createFunctionEvent);
+          }
+        }
       }
     }
 
@@ -6216,20 +6216,33 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       boolean success = false;
       Function func = null;
       RawStore ms = getMS();
-
       try {
         ms.openTransaction();
-
         func = ms.getFunction(dbName, funcName);
         if (func == null) {
           throw new NoSuchObjectException("Function " + funcName + " does not exist");
         }
+
         ms.dropFunction(dbName, funcName);
+        if (transactionalListeners.size() > 0) {
+          DropFunctionEvent dropFunctionEvent = new DropFunctionEvent(func, true, this);
+          for (MetaStoreEventListener transactionalListener : transactionalListeners) {
+            transactionalListener.onDropFunction(dropFunctionEvent);
+          }
+        }
+
         success = ms.commitTransaction();
       } finally {
         if (!success) {
           ms.rollbackTransaction();
         }
+
+        if (listeners.size() > 0) {
+          DropFunctionEvent dropFunctionEvent = new DropFunctionEvent(func, success, this);
+          for (MetaStoreEventListener listener : listeners) {
+            listener.onDropFunction(dropFunctionEvent);
+          }
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
index 5e46ae1..b0defb5 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
@@ -28,8 +28,10 @@ import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
@@ -145,6 +147,20 @@ public abstract class MetaStoreEventListener implements Configurable {
   }
 
   /**
+   * @param fnEvent function event
+   * @throws MetaException
+   */
+  public void onCreateFunction (CreateFunctionEvent fnEvent) throws MetaException {
+  }
+
+  /**
+   * @param fnEvent function event
+   * @throws MetaException
+   */
+  public void onDropFunction (DropFunctionEvent fnEvent) throws MetaException {
+  }
+
+  /**
    * This will be called when an insert is executed that does not cause a partition to be added.
    * If an insert causes a partition to be added it will cause {@link #onAddPartition} to be
    * called instead.

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java
new file mode 100644
index 0000000..717ede2
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/events/CreateFunctionEvent.java
@@ -0,0 +1,39 @@
+/**
+ * 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.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.api.Function;
+
+public class CreateFunctionEvent extends ListenerEvent {
+
+  private final Function function;
+
+  public CreateFunctionEvent (Function function, boolean status, HMSHandler handler) {
+    super (status, handler);
+    this.function = function;
+  }
+
+  /**
+   * @return the function
+   */
+  public Function getFunction () {
+    return function;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java b/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java
new file mode 100644
index 0000000..7190aae
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/events/DropFunctionEvent.java
@@ -0,0 +1,39 @@
+/**
+ * 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.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.api.Function;
+
+public class DropFunctionEvent extends ListenerEvent {
+
+  private final Function function;
+
+  public DropFunctionEvent(Function function, boolean status, HMSHandler handler) {
+    super(status, handler);
+    this.function = function;
+  }
+
+  /**
+   * @return the function
+   */
+  public Function getFunction() {
+    return function;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/62d802b8/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java
index a3b16d0..182e724 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyListener.java
@@ -30,8 +30,10 @@ import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
 import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
@@ -126,6 +128,16 @@ public class DummyListener extends MetaStoreEventListener{
     addEvent(indexEvent);
   }
 
+  @Override
+  public void onCreateFunction (CreateFunctionEvent fnEvent) throws MetaException {
+    addEvent(fnEvent);
+  }
+
+  @Override
+  public void onDropFunction (DropFunctionEvent fnEvent) throws MetaException {
+    addEvent(fnEvent);
+  }
+
   private void addEvent(ListenerEvent event) {
     notifyList.add(event);
   }


[16/35] hive git commit: HIVE-15233: UDF UUID() should be non-deterministic (Chao Sun, reviewed by Xuefu Zhang)

Posted by se...@apache.org.
HIVE-15233: UDF UUID() should be non-deterministic (Chao Sun, reviewed by Xuefu Zhang)


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

Branch: refs/heads/hive-14535
Commit: aebc9060bcf097407005997502e065feb7007c80
Parents: 62d802b
Author: Chao Sun <su...@apache.org>
Authored: Fri Nov 18 09:24:42 2016 -0800
Committer: Chao Sun <su...@apache.org>
Committed: Fri Nov 18 09:24:42 2016 -0800

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/udf/UDFUUID.java     | 1 +
 ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java | 5 +++++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/aebc9060/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFUUID.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFUUID.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFUUID.java
index 1340ded..4999bb0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFUUID.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFUUID.java
@@ -36,6 +36,7 @@ extended = "The value is returned as a canonical UUID 36-character string.\n"
 + "  '0baf1f52-53df-487f-8292-99a03716b688'\n"
 + "  > SELECT _FUNC_();\n"
 + "  '36718a53-84f5-45d6-8796-4f79983ad49d'")
+@UDFType(deterministic = false)
 public class UDFUUID extends UDF {
   private final Text result = new Text();
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/aebc9060/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java
index 5f04547..cf32c11 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFUUID.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hive.ql.udf;
 
 import junit.framework.TestCase;
 
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.junit.Test;
 
 public class TestUDFUUID extends TestCase {
@@ -34,5 +36,8 @@ public class TestUDFUUID extends TestCase {
     
     assertEquals(id1.length(), 36);
     assertEquals(id2.length(), 36);
+
+    GenericUDFBridge bridge = new GenericUDFBridge("uuid", false, UDFUUID.class.getName());
+    assertFalse(FunctionRegistry.isDeterministic(bridge));
   }
 }


[05/35] hive git commit: HIVE-11208: Can not drop a default partition __HIVE_DEFAULT_PARTITION__ which is not a "string" type (Aihua Xu, reviewed by Yongzhi Chen)

Posted by se...@apache.org.
HIVE-11208: Can not drop a default partition __HIVE_DEFAULT_PARTITION__ which is not a "string" type (Aihua Xu, reviewed by Yongzhi Chen)


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

Branch: refs/heads/hive-14535
Commit: 2de8b3192306376a382f6b212f24fde3f990e57d
Parents: 085650e
Author: Aihua Xu <ai...@apache.org>
Authored: Fri Mar 4 17:03:18 2016 -0500
Committer: Aihua Xu <ai...@apache.org>
Committed: Thu Nov 17 12:00:27 2016 -0500

----------------------------------------------------------------------
 .../exec/ExprNodeConstantDefaultEvaluator.java  |  50 ++++++
 .../hive/ql/exec/ExprNodeEvaluatorFactory.java  |   7 +
 .../ql/optimizer/ppr/PartExprEvalUtils.java     |   8 +-
 .../hive/ql/optimizer/ppr/PartitionPruner.java  |  19 ++-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |  23 ++-
 .../ql/plan/ExprNodeConstantDefaultDesc.java    |  86 ++++++++++
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |   9 +
 .../ql/udf/generic/GenericUDFOPNotEqual.java    |   9 +
 .../clientpositive/drop_partitions_filter4.q    |  29 ++++
 .../drop_partitions_filter4.q.out               | 167 +++++++++++++++++++
 10 files changed, 390 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantDefaultEvaluator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantDefaultEvaluator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantDefaultEvaluator.java
new file mode 100644
index 0000000..89a75eb
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantDefaultEvaluator.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec;
+
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+/**
+ * ExprNodeConstantEvaluator.
+ *
+ */
+public class ExprNodeConstantDefaultEvaluator extends ExprNodeEvaluator<ExprNodeConstantDefaultDesc> {
+
+  transient ObjectInspector writableObjectInspector;
+
+  public ExprNodeConstantDefaultEvaluator(ExprNodeConstantDefaultDesc expr) {
+    super(expr);
+    writableObjectInspector = expr.getWritableObjectInspector();
+  }
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector rowInspector) throws HiveException {
+    return writableObjectInspector;
+  }
+
+  @Override
+  protected Object _evaluate(Object row, int version) throws HiveException {
+    return expr;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
index 5a532c4..0d03d8f 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
@@ -42,6 +43,12 @@ public final class ExprNodeEvaluatorFactory {
     if (desc instanceof ExprNodeConstantDesc) {
       return new ExprNodeConstantEvaluator((ExprNodeConstantDesc) desc);
     }
+
+    // Special 'default' constant node
+    if (desc instanceof ExprNodeConstantDefaultDesc) {
+      return new ExprNodeConstantDefaultEvaluator((ExprNodeConstantDefaultDesc) desc);
+    }
+
     // Column-reference node, e.g. a column in the input row
     if (desc instanceof ExprNodeColumnDesc) {
       return new ExprNodeColumnEvaluator((ExprNodeColumnDesc) desc);

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
index dc5d2df..1103d35 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
@@ -104,16 +104,16 @@ public class PartExprEvalUtils {
   }
 
   static synchronized public ObjectPair<PrimitiveObjectInspector, ExprNodeEvaluator> prepareExpr(
-      ExprNodeGenericFuncDesc expr, List<String> partNames,
+      ExprNodeGenericFuncDesc expr, List<String> partColumnNames,
       List<PrimitiveTypeInfo> partColumnTypeInfos) throws HiveException {
     // Create the row object
     List<ObjectInspector> partObjectInspectors = new ArrayList<ObjectInspector>();
-    for (int i = 0; i < partNames.size(); i++) {
+    for (int i = 0; i < partColumnNames.size(); i++) {
       partObjectInspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(
-          partColumnTypeInfos.get(i)));
+        partColumnTypeInfos.get(i)));
     }
     StructObjectInspector objectInspector = ObjectInspectorFactory
-        .getStandardStructObjectInspector(partNames, partObjectInspectors);
+        .getStandardStructObjectInspector(partColumnNames, partObjectInspectors);
 
     ExprNodeEvaluator evaluator = ExprNodeEvaluatorFactory.get(expr);
     ObjectInspector evaluateResultOI = evaluator.initialize(objectInspector);

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
index 26e936e..9d7307e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
@@ -62,6 +63,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
@@ -542,11 +544,18 @@ public class PartitionPruner extends Transform {
 
       ArrayList<Object> convertedValues = new ArrayList<Object>(values.size());
       for(int i=0; i<values.size(); i++) {
-        Object o = ObjectInspectorConverters.getConverter(
-            PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-            PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(partColumnTypeInfos.get(i)))
-            .convert(values.get(i));
-        convertedValues.add(o);
+        String partitionValue = values.get(i);
+        PrimitiveTypeInfo typeInfo = partColumnTypeInfos.get(i);
+
+        if (partitionValue.equals(defaultPartitionName)) {
+          convertedValues.add(new ExprNodeConstantDefaultDesc(typeInfo, defaultPartitionName));
+        } else {
+          Object o = ObjectInspectorConverters.getConverter(
+              PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+              PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(typeInfo))
+              .convert(partitionValue);
+          convertedValues.add(o);
+        }
       }
 
       // Evaluate the expression tree.

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index a264c4d..c7389a8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -101,6 +101,7 @@ import org.apache.hadoop.hive.ql.plan.DropIndexDesc;
 import org.apache.hadoop.hive.ql.plan.DropTableDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.FetchWork;
@@ -3093,6 +3094,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
    */
   private Map<Integer, List<ExprNodeGenericFuncDesc>> getFullPartitionSpecs(
       CommonTree ast, Table tab, boolean canGroupExprs) throws SemanticException {
+    String defaultPartitionName = HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME);
     Map<String, String> colTypes = new HashMap<String, String>();
     for (FieldSchema fs : tab.getPartitionKeys()) {
       colTypes.put(fs.getName().toLowerCase(), fs.getType());
@@ -3114,23 +3116,28 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         TypeCheckCtx typeCheckCtx = new TypeCheckCtx(null);
         ExprNodeConstantDesc valExpr = (ExprNodeConstantDesc)TypeCheckProcFactory
             .genExprNode(partValNode, typeCheckCtx).get(partValNode);
+        Object val = valExpr.getValue();
+
+        boolean isDefaultPartitionName =  val.equals(defaultPartitionName);
 
         String type = colTypes.get(key);
+        PrimitiveTypeInfo pti = TypeInfoFactory.getPrimitiveTypeInfo(type);
         if (type == null) {
           throw new SemanticException("Column " + key + " not found");
         }
         // Create the corresponding hive expression to filter on partition columns.
-        PrimitiveTypeInfo pti = TypeInfoFactory.getPrimitiveTypeInfo(type);
-        Object val = valExpr.getValue();
-        if (!valExpr.getTypeString().equals(type)) {
-          Converter converter = ObjectInspectorConverters.getConverter(
-            TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(valExpr.getTypeInfo()),
-            TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(pti));
-          val = converter.convert(valExpr.getValue());
+        if (!isDefaultPartitionName) {
+          if (!valExpr.getTypeString().equals(type)) {
+            Converter converter = ObjectInspectorConverters.getConverter(
+              TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(valExpr.getTypeInfo()),
+              TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(pti));
+            val = converter.convert(valExpr.getValue());
+          }
         }
+
         ExprNodeColumnDesc column = new ExprNodeColumnDesc(pti, key, null, true);
         ExprNodeGenericFuncDesc op = makeBinaryPredicate(operator, column,
-            new ExprNodeConstantDesc(pti, val));
+            isDefaultPartitionName ? new ExprNodeConstantDefaultDesc(pti, defaultPartitionName) : new ExprNodeConstantDesc(pti, val));
         // If it's multi-expr filter (e.g. a='5', b='2012-01-02'), AND with previous exprs.
         expr = (expr == null) ? op : makeBinaryPredicate("and", expr, op);
         names.add(key);

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDefaultDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDefaultDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDefaultDesc.java
new file mode 100644
index 0000000..8b3c0bc
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDefaultDesc.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.plan;
+
+import java.io.Serializable;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * A constant expression with default value and data type. The value is different
+ * from any value of that data type. Used to represent the default partition in
+ * the expression of x =/!= __HIVE_DEFAULT_PARTITION__
+ */
+public class ExprNodeConstantDefaultDesc extends ExprNodeDesc implements Serializable {
+  private static final long serialVersionUID = 1L;
+  private final Object value;     // The internal value for the default
+
+  public ExprNodeConstantDefaultDesc() {
+    value = null;
+  }
+
+  public ExprNodeConstantDefaultDesc(TypeInfo typeInfo, Object value) {
+    super(typeInfo);
+    this.value = value;
+  }
+
+  @Override
+  public String toString() {
+    return "Const " + typeInfo.toString() + " default";
+  }
+
+  @Override
+  public String getExprString() {
+    return value == null ? "null" : value.toString();
+  }
+
+  @Override
+  public boolean isSame(Object o) {
+    if (!(o instanceof ExprNodeConstantDefaultDesc)) {
+      return false;
+    }
+    ExprNodeConstantDefaultDesc dest = (ExprNodeConstantDefaultDesc) o;
+    if (!typeInfo.equals(dest.getTypeInfo())) {
+      return false;
+    }
+    if (value == null) {
+      if (dest.value != null) {
+        return false;
+      }
+    } else if (!value.equals(dest.value)) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public ExprNodeDesc clone() {
+    return new ExprNodeConstantDefaultDesc(typeInfo, value);
+  }
+
+  @Override
+  public int hashCode() {
+    int superHashCode = super.hashCode();
+    HashCodeBuilder builder = new HashCodeBuilder();
+    builder.appendSuper(superHashCode);
+    return builder.toHashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
index b393843..0002b07 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarEqualLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 
 /**
@@ -109,6 +110,14 @@ public class GenericUDFOPEqual extends GenericUDFBaseCompare {
       return null;
     }
 
+    // Handle 'default' constant which has a data type with special value
+    if (o0 instanceof ExprNodeConstantDefaultDesc || o1 instanceof ExprNodeConstantDefaultDesc) {
+      ExprNodeConstantDefaultDesc default0 = o0 instanceof ExprNodeConstantDefaultDesc ? (ExprNodeConstantDefaultDesc)o0 : null;
+      ExprNodeConstantDefaultDesc default1 = o1 instanceof ExprNodeConstantDefaultDesc ? (ExprNodeConstantDefaultDesc)o1 : null;
+      result.set(default0 != null && default1 != null && default0.isSame(default1));
+      return result;
+    }
+
     switch(compareType) {
     case COMPARE_TEXT:
       result.set(soi0.getPrimitiveWritableObject(o0).equals(

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
index ed6aa36..9652859 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongScal
 import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarNotEqualLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 
 /**
@@ -109,6 +110,14 @@ public class GenericUDFOPNotEqual extends GenericUDFBaseCompare {
       return null;
     }
 
+    // Handle 'default' constant which has a data type with special value
+    if (o0 instanceof ExprNodeConstantDefaultDesc || o1 instanceof ExprNodeConstantDefaultDesc) {
+      ExprNodeConstantDefaultDesc default0 = o0 instanceof ExprNodeConstantDefaultDesc ? (ExprNodeConstantDefaultDesc)o0 : null;
+      ExprNodeConstantDefaultDesc default1 = o1 instanceof ExprNodeConstantDefaultDesc ? (ExprNodeConstantDefaultDesc)o1 : null;
+      result.set(default0 == null || default1 == null || !default0.isSame(default1));
+      return result;
+    }
+
     switch(compareType) {
     case COMPARE_TEXT:
       result.set(!soi0.getPrimitiveWritableObject(o0).equals(

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/test/queries/clientpositive/drop_partitions_filter4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/drop_partitions_filter4.q b/ql/src/test/queries/clientpositive/drop_partitions_filter4.q
new file mode 100644
index 0000000..f0e5e19
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/drop_partitions_filter4.q
@@ -0,0 +1,29 @@
+SET hive.exec.dynamic.partition.mode=nonstrict;
+
+create table ptestfilter (a string, b int) partitioned by (c double);
+INSERT OVERWRITE TABLE ptestfilter PARTITION (c) select 'Col1', 1, null;
+alter table ptestfilter add partition (c=3.4);
+alter table ptestfilter add partition (c=5.55);
+show partitions ptestfilter;
+
+alter table ptestfilter drop partition(c = '__HIVE_DEFAULT_PARTITION__');
+alter table ptestfilter drop partition(c = 3.40);
+show partitions ptestfilter;
+
+drop table ptestfilter;
+
+create table ptestfilter (a string, b int) partitioned by (c string, d int);
+INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col1', 1, null, null;
+INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col2', 2, null, 2;
+INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col3', 3, 'Uganda', null;
+alter table ptestfilter add partition (c='Germany', d=2);
+show partitions ptestfilter;
+
+alter table ptestfilter drop partition (c='__HIVE_DEFAULT_PARTITION__');
+alter table ptestfilter drop partition (c='Uganda', d='__HIVE_DEFAULT_PARTITION__');
+alter table ptestfilter drop partition (c='Germany', d=2);
+show partitions ptestfilter;
+
+drop table ptestfilter;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/test/results/clientpositive/drop_partitions_filter4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/drop_partitions_filter4.q.out b/ql/src/test/results/clientpositive/drop_partitions_filter4.q.out
new file mode 100644
index 0000000..0e6d41a
--- /dev/null
+++ b/ql/src/test/results/clientpositive/drop_partitions_filter4.q.out
@@ -0,0 +1,167 @@
+PREHOOK: query: create table ptestfilter (a string, b int) partitioned by (c double)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: create table ptestfilter (a string, b int) partitioned by (c double)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ptestfilter
+PREHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c) select 'Col1', 1, null
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c) select 'Col1', 1, null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__).a SIMPLE []
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__).b SIMPLE []
+PREHOOK: query: alter table ptestfilter add partition (c=3.4)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: alter table ptestfilter add partition (c=3.4)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=3.4
+PREHOOK: query: alter table ptestfilter add partition (c=5.55)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: alter table ptestfilter add partition (c=5.55)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=5.55
+PREHOOK: query: show partitions ptestfilter
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ptestfilter
+POSTHOOK: query: show partitions ptestfilter
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ptestfilter
+c=3.4
+c=5.55
+c=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition(c = '__HIVE_DEFAULT_PARTITION__')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: query: alter table ptestfilter drop partition(c = '__HIVE_DEFAULT_PARTITION__')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition(c = 3.40)
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=3.4
+POSTHOOK: query: alter table ptestfilter drop partition(c = 3.40)
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=3.4
+PREHOOK: query: show partitions ptestfilter
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ptestfilter
+POSTHOOK: query: show partitions ptestfilter
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ptestfilter
+c=5.55
+PREHOOK: query: drop table ptestfilter
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: drop table ptestfilter
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter
+PREHOOK: query: create table ptestfilter (a string, b int) partitioned by (c string, d int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: create table ptestfilter (a string, b int) partitioned by (c string, d int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ptestfilter
+PREHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col1', 1, null, null
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col1', 1, null, null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__,d=__HIVE_DEFAULT_PARTITION__).a SIMPLE []
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__,d=__HIVE_DEFAULT_PARTITION__).b SIMPLE []
+PREHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col2', 2, null, 2
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col2', 2, null, 2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=2
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__,d=2).a SIMPLE []
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__,d=2).b SIMPLE []
+PREHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col3', 3, 'Uganda', null
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col3', 3, 'Uganda', null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ptestfilter@c=Uganda/d=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: ptestfilter PARTITION(c=Uganda,d=__HIVE_DEFAULT_PARTITION__).a SIMPLE []
+POSTHOOK: Lineage: ptestfilter PARTITION(c=Uganda,d=__HIVE_DEFAULT_PARTITION__).b SIMPLE []
+PREHOOK: query: alter table ptestfilter add partition (c='Germany', d=2)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: alter table ptestfilter add partition (c='Germany', d=2)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=Germany/d=2
+PREHOOK: query: show partitions ptestfilter
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ptestfilter
+POSTHOOK: query: show partitions ptestfilter
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ptestfilter
+c=Germany/d=2
+c=Uganda/d=__HIVE_DEFAULT_PARTITION__
+c=__HIVE_DEFAULT_PARTITION__/d=2
+c=__HIVE_DEFAULT_PARTITION__/d=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition (c='__HIVE_DEFAULT_PARTITION__')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=2
+PREHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: query: alter table ptestfilter drop partition (c='__HIVE_DEFAULT_PARTITION__')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=2
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition (c='Uganda', d='__HIVE_DEFAULT_PARTITION__')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=Uganda/d=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: query: alter table ptestfilter drop partition (c='Uganda', d='__HIVE_DEFAULT_PARTITION__')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=Uganda/d=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition (c='Germany', d=2)
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=Germany/d=2
+POSTHOOK: query: alter table ptestfilter drop partition (c='Germany', d=2)
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=Germany/d=2
+PREHOOK: query: show partitions ptestfilter
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ptestfilter
+POSTHOOK: query: show partitions ptestfilter
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ptestfilter
+PREHOOK: query: drop table ptestfilter
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: drop table ptestfilter
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter