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 2017/02/24 01:08:33 UTC

[01/50] [abbrv] hive git commit: HIVE-15161: migrate ColumnStats to use jackson (Zoltan Haindrich, via Pengcheng Xiong)

Repository: hive
Updated Branches:
  refs/heads/hive-14535 74d93333c -> 2014ece97


HIVE-15161: migrate ColumnStats to use jackson (Zoltan Haindrich, via Pengcheng Xiong)


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

Branch: refs/heads/hive-14535
Commit: 6e652a3b990bc53e61970ddc1aa2c0b503cd13be
Parents: e732aa2
Author: Pengcheng Xiong <px...@apache.org>
Authored: Thu Feb 16 10:13:56 2017 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Thu Feb 16 10:13:56 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hive/common/StatsSetupConst.java     | 223 ++++++++++---------
 .../hadoop/hive/common/TestStatsSetupConst.java |  54 +++++
 2 files changed, 171 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6e652a3b/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java b/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
index c78f005..926b4a6 100644
--- a/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
+++ b/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
@@ -17,19 +17,31 @@
  */
 package org.apache.hadoop.hive.common;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.json.JSONException;
-import org.json.JSONObject;
-
-import java.util.LinkedHashMap;
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+
 
 /**
  * A class that defines the constant strings used by the statistics implementation.
@@ -144,35 +156,62 @@ public class StatsSetupConst {
   public static final String[] TABLE_PARAMS_STATS_KEYS = new String[] {
     COLUMN_STATS_ACCURATE, NUM_FILES, TOTAL_SIZE,ROW_COUNT, RAW_DATA_SIZE, NUM_PARTITIONS};
 
+  private static class ColumnStatsAccurate {
+    private static ObjectReader objectReader;
+    private static ObjectWriter objectWriter;
+
+    static {
+      ObjectMapper objectMapper = new ObjectMapper();
+      objectReader = objectMapper.readerFor(ColumnStatsAccurate.class);
+      objectWriter = objectMapper.writerFor(ColumnStatsAccurate.class);
+    }
+
+    static class BooleanSerializer extends JsonSerializer<Boolean> {
+
+      @Override
+      public void serialize(Boolean value, JsonGenerator jsonGenerator,
+          SerializerProvider serializerProvider) throws IOException, JsonProcessingException {
+        jsonGenerator.writeString(value.toString());
+      }
+    }
+
+    static class BooleanDeserializer extends JsonDeserializer<Boolean> {
+
+      public Boolean deserialize(JsonParser jsonParser,
+          DeserializationContext deserializationContext)
+              throws IOException, JsonProcessingException {
+        return Boolean.valueOf(jsonParser.getValueAsString());
+      }
+    }
+
+    @JsonInclude(JsonInclude.Include.NON_DEFAULT)
+    @JsonSerialize(using = BooleanSerializer.class)
+    @JsonDeserialize(using = BooleanDeserializer.class)
+    @JsonProperty(BASIC_STATS)
+    boolean basicStats;
+
+    @JsonInclude(JsonInclude.Include.NON_EMPTY)
+    @JsonProperty(COLUMN_STATS)
+    @JsonSerialize(contentUsing = BooleanSerializer.class)
+    @JsonDeserialize(contentUsing = BooleanDeserializer.class)
+    TreeMap<String, Boolean> columnStats = new TreeMap<>();
+
+  };
+
   public static boolean areBasicStatsUptoDate(Map<String, String> params) {
-    JSONObject stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
-    if (stats != null && stats.has(BASIC_STATS)) {
-      return true;
-    } else {
+    if (params == null) {
       return false;
     }
+    ColumnStatsAccurate stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
+    return stats.basicStats;
   }
 
   public static boolean areColumnStatsUptoDate(Map<String, String> params, String colName) {
-    JSONObject stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
-    try {
-      if (!stats.has(COLUMN_STATS)) {
-        return false;
-      } else {
-        JSONObject columns = stats.getJSONObject(COLUMN_STATS);
-        if (columns != null && columns.has(colName)) {
-          return true;
-        } else {
-          return false;
-        }
-      }
-    } catch (JSONException e) {
-      // For backward compatibility, if previous value can not be parsed to a
-      // json object, it will come here.
-      LOG.debug("In StatsSetupConst, JsonParser can not parse COLUMN_STATS.");
+    if (params == null) {
       return false;
     }
-
+    ColumnStatsAccurate stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
+    return stats.columnStats.containsKey(colName);
   }
 
   // It will only throw JSONException when stats.put(BASIC_STATS, TRUE)
@@ -180,79 +219,67 @@ public class StatsSetupConst {
   // note that set basic stats false will wipe out column stats too.
   public static void setBasicStatsState(Map<String, String> params, String setting) {
     if (setting.equals(FALSE)) {
-      if (params != null && params.containsKey(COLUMN_STATS_ACCURATE)) {
+      if (params!=null && params.containsKey(COLUMN_STATS_ACCURATE)) {
         params.remove(COLUMN_STATS_ACCURATE);
       }
-    } else {
-      JSONObject stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
-      
-      try {
-        stats.put(BASIC_STATS, TRUE);
-      } catch (JSONException e) {
-        // impossible to throw any json exceptions.
-        LOG.trace(e.getMessage());
-      }
-      params.put(COLUMN_STATS_ACCURATE, stats.toString());
+      return;
+    }
+    if (params == null) {
+      throw new RuntimeException("params are null...cant set columnstatstate!");
+    }
+    ColumnStatsAccurate stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
+    stats.basicStats = true;
+    try {
+      params.put(COLUMN_STATS_ACCURATE, ColumnStatsAccurate.objectWriter.writeValueAsString(stats));
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException("can't serialize column stats", e);
     }
   }
 
   public static void setColumnStatsState(Map<String, String> params, List<String> colNames) {
-    try {
-      JSONObject stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
+    if (params == null) {
+      throw new RuntimeException("params are null...cant set columnstatstate!");
+    }
+    ColumnStatsAccurate stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
 
-      JSONObject colStats;
-      if (stats.has(COLUMN_STATS)) {
-        colStats = stats.getJSONObject(COLUMN_STATS);
-      } else {
-        colStats = new JSONObject(new TreeMap<String,String>());
-      }
-      for (String colName : colNames) {
-        if (!colStats.has(colName)) {
-          colStats.put(colName, TRUE);
-        }
+    for (String colName : colNames) {
+      if (!stats.columnStats.containsKey(colName)) {
+        stats.columnStats.put(colName, true);
       }
-      stats.put(COLUMN_STATS, colStats);
-      params.put(COLUMN_STATS_ACCURATE, stats.toString());
-    } catch (JSONException e) {
-      // impossible to throw any json exceptions.
+    }
+    try {
+      params.put(COLUMN_STATS_ACCURATE, ColumnStatsAccurate.objectWriter.writeValueAsString(stats));
+    } catch (JsonProcessingException e) {
       LOG.trace(e.getMessage());
     }
   }
 
   public static void clearColumnStatsState(Map<String, String> params) {
-    String statsAcc;
-    if (params != null && (statsAcc = params.get(COLUMN_STATS_ACCURATE)) != null) {
-      // statsAcc may not be jason format, which will throw exception
-      JSONObject stats = parseStatsAcc(statsAcc);
-      
-      if (stats.has(COLUMN_STATS)) {
-        stats.remove(COLUMN_STATS);
-      }
-      params.put(COLUMN_STATS_ACCURATE, stats.toString());
+    if (params == null) {
+      return;
+    }
+    ColumnStatsAccurate stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
+    stats.columnStats.clear();
+
+    try {
+      params.put(COLUMN_STATS_ACCURATE, ColumnStatsAccurate.objectWriter.writeValueAsString(stats));
+    } catch (JsonProcessingException e) {
+      LOG.trace(e.getMessage());
     }
   }
 
   public static void removeColumnStatsState(Map<String, String> params, List<String> colNames) {
-    String statsAcc;
-    if (params != null && (statsAcc = params.get(COLUMN_STATS_ACCURATE)) != null) {
-      // statsAcc may not be jason format, which will throw exception
-      JSONObject stats = parseStatsAcc(statsAcc);
-      try {
-        JSONObject colStats = stats.getJSONObject(COLUMN_STATS);
-        for (String colName : colNames) {
-          if (colStats.has(colName)) {
-            colStats.remove(colName);
-          }
-        }
-        if (colStats.length() != 0) {
-          stats.put(COLUMN_STATS, colStats);
-        } else {
-          stats.remove(COLUMN_STATS);
-        }
-        params.put(COLUMN_STATS_ACCURATE, stats.toString());
-      } catch (JSONException e) {
-        LOG.debug(e.getMessage());
+    if (params == null) {
+      return;
+    }
+    try {
+      ColumnStatsAccurate stats = parseStatsAcc(params.get(COLUMN_STATS_ACCURATE));
+      for (String string : colNames) {
+        stats.columnStats.remove(string);
       }
+      params.put(COLUMN_STATS_ACCURATE, ColumnStatsAccurate.objectWriter.writeValueAsString(stats));
+    } catch (JsonProcessingException e) {
+      LOG.trace(e.getMessage());
     }
   }
 
@@ -265,34 +292,18 @@ public class StatsSetupConst {
     setBasicStatsState(params, setting);
   }
   
-  private static JSONObject parseStatsAcc(String statsAcc) {
+  private static ColumnStatsAccurate parseStatsAcc(String statsAcc) {
     if (statsAcc == null) {
-      return new JSONObject(new LinkedHashMap<String,Object>());
-    } else {
-      try {
-        return new JSONObject(statsAcc);
-      } catch (JSONException e) {
-        return statsAccUpgrade(statsAcc);
-      }
+      return new ColumnStatsAccurate();
     }
-  }
-
-  private static JSONObject statsAccUpgrade(String statsAcc) {
-    JSONObject stats;
-    // old format of statsAcc, e.g., TRUE or FALSE
-    LOG.debug("In StatsSetupConst, JsonParser can not parse statsAcc.");
-    stats = new JSONObject(new LinkedHashMap<String,Object>());
     try {
-      if (statsAcc.equals(TRUE)) {
-        stats.put(BASIC_STATS, TRUE);
-      } else {
-        stats.put(BASIC_STATS, FALSE);
+      return ColumnStatsAccurate.objectReader.readValue(statsAcc);
+    } catch (Exception e) {
+      ColumnStatsAccurate ret = new ColumnStatsAccurate();
+      if (TRUE.equalsIgnoreCase(statsAcc)) {
+        ret.basicStats = true;
       }
-    } catch (JSONException e1) {
-      // impossible to throw any json exceptions.
-      LOG.trace(e1.getMessage());
+      return ret;
     }
-    return stats;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/6e652a3b/common/src/test/org/apache/hadoop/hive/common/TestStatsSetupConst.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/TestStatsSetupConst.java b/common/src/test/org/apache/hadoop/hive/common/TestStatsSetupConst.java
index 7a7ad42..792b862 100644
--- a/common/src/test/org/apache/hadoop/hive/common/TestStatsSetupConst.java
+++ b/common/src/test/org/apache/hadoop/hive/common/TestStatsSetupConst.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.common;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -53,4 +54,57 @@ public class TestStatsSetupConst {
     assertEquals("{\"BASIC_STATS\":\"true\"}",params.get(StatsSetupConst.COLUMN_STATS_ACCURATE));
   }
 
+  @Test
+  public void testSetBasicStatsState_falseIsAbsent() {
+    Map<String, String> params=new HashMap<>();
+    StatsSetupConst.setBasicStatsState(params, String.valueOf(true));
+    StatsSetupConst.setBasicStatsState(params, String.valueOf(false));
+    assertNull(params.get(StatsSetupConst.COLUMN_STATS_ACCURATE));
+  }
+
+  // earlier implementation have quoted boolean values...so the new implementation should preserve this
+  @Test
+  public void testStatColumnEntriesCompat() {
+    Map<String, String> params0=new HashMap<>();
+    StatsSetupConst.setBasicStatsState(params0, String.valueOf(true));
+    StatsSetupConst.setColumnStatsState(params0, Lists.newArrayList("Foo"));
+
+    assertEquals("{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"Foo\":\"true\"}}",params0.get(StatsSetupConst.COLUMN_STATS_ACCURATE));
+  }
+
+  @Test
+  public void testColumnEntries_orderIndependence() {
+    Map<String, String> params0=new HashMap<>();
+    StatsSetupConst.setBasicStatsState(params0, String.valueOf(true));
+    StatsSetupConst.setColumnStatsState(params0, Lists.newArrayList("Foo","Bar"));
+    Map<String, String> params1=new HashMap<>();
+    StatsSetupConst.setColumnStatsState(params1, Lists.newArrayList("Bar","Foo"));
+    StatsSetupConst.setBasicStatsState(params1, String.valueOf(true));
+
+    assertEquals(params0.get(StatsSetupConst.COLUMN_STATS_ACCURATE),params1.get(StatsSetupConst.COLUMN_STATS_ACCURATE));
+  }
+
+  @Test
+  public void testColumnEntries_orderIndependence2() {
+    Map<String, String> params0=new HashMap<>();
+    // in case jackson is able to deserialize...it may use a different implementation for the map - which may not preserve order
+    StatsSetupConst.setBasicStatsState(params0, String.valueOf(true));
+    StatsSetupConst.setColumnStatsState(params0, Lists.newArrayList("year"));
+    StatsSetupConst.setColumnStatsState(params0, Lists.newArrayList("year","month"));
+    Map<String, String> params1=new HashMap<>();
+    StatsSetupConst.setColumnStatsState(params1, Lists.newArrayList("month","year"));
+    StatsSetupConst.setBasicStatsState(params1, String.valueOf(true));
+
+    System.out.println(params0.get(StatsSetupConst.COLUMN_STATS_ACCURATE));
+    assertEquals(params0.get(StatsSetupConst.COLUMN_STATS_ACCURATE),params1.get(StatsSetupConst.COLUMN_STATS_ACCURATE));
+  }
+
+  // FIXME: current objective is to keep the previous outputs...but this is possibly bad..
+  @Test
+  public void testColumnEntries_areKept_whenBasicIsAbsent() {
+    Map<String, String> params=new HashMap<>();
+    StatsSetupConst.setBasicStatsState(params, String.valueOf(false));
+    StatsSetupConst.setColumnStatsState(params, Lists.newArrayList("Foo"));
+    assertEquals("{\"COLUMN_STATS\":{\"Foo\":\"true\"}}",params.get(StatsSetupConst.COLUMN_STATS_ACCURATE));
+  }
 }


[35/50] [abbrv] hive git commit: HIVE-15928: Parallelization of Select queries in Druid handler (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-15928: Parallelization of Select queries in Druid handler (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/8ab1889d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8ab1889d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8ab1889d

Branch: refs/heads/hive-14535
Commit: 8ab1889dd9afe958e96cc62fc973771f61cadcba
Parents: 8973d2c
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Thu Feb 16 14:40:41 2017 +0000
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Feb 22 10:17:28 2017 +0000

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  10 +-
 .../druid/io/DruidQueryBasedInputFormat.java    | 124 ++++++++++++++++---
 .../hadoop/hive/druid/io/HiveDruidSplit.java    |  30 ++---
 .../druid/serde/DruidQueryRecordReader.java     |   3 +-
 .../TestHiveDruidQueryBasedInputFormat.java     |  21 ++--
 5 files changed, 132 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8ab1889d/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 7c88f4f..3777fa9 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1933,12 +1933,20 @@ public class HiveConf extends Configuration {
     HIVE_DRUID_COORDINATOR_DEFAULT_ADDRESS("hive.druid.coordinator.address.default", "localhost:8081",
             "Address of the Druid coordinator. It is used to check the load status of newly created segments"
     ),
+    HIVE_DRUID_SELECT_DISTRIBUTE("hive.druid.select.distribute", true,
+        "If it is set to true, we distribute the execution of Druid Select queries. Concretely, we retrieve\n" +
+        "the result for Select queries directly from the Druid nodes containing the segments data.\n" +
+        "In particular, first we contact the Druid broker node to obtain the nodes containing the segments\n" +
+        "for the given query, and then we contact those nodes to retrieve the results for the query.\n" +
+        "If it is set to false, we do not execute the Select queries in a distributed fashion. Instead, results\n" +
+        "for those queries are returned by the Druid broker node."),
     HIVE_DRUID_SELECT_THRESHOLD("hive.druid.select.threshold", 10000,
+        "Takes only effect when hive.druid.select.distribute is set to false. \n" +
         "When we can split a Select query, this is the maximum number of rows that we try to retrieve\n" +
         "per query. In order to do that, we obtain the estimated size for the complete result. If the\n" +
         "number of records of the query results is larger than this threshold, we split the query in\n" +
         "total number of rows/threshold parts across the time dimension. Note that we assume the\n" +
-        "records to be split uniformly across the time dimension"),
+        "records to be split uniformly across the time dimension."),
     HIVE_DRUID_NUM_HTTP_CONNECTION("hive.druid.http.numConnection", 20, "Number of connections used by\n" +
         "the HTTP client."),
     HIVE_DRUID_HTTP_READ_TIMEOUT("hive.druid.http.read.timeout", "PT1M", "Read timeout period for the HTTP\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/8ab1889d/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java
index 8b37840..0b35428 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.druid.io;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -51,6 +52,7 @@ import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.jboss.netty.handler.codec.http.HttpMethod;
 import org.joda.time.Interval;
 import org.joda.time.Period;
 import org.joda.time.chrono.ISOChronology;
@@ -60,23 +62,28 @@ import org.slf4j.LoggerFactory;
 import com.fasterxml.jackson.core.JsonParseException;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.JsonMappingException;
+import com.google.common.collect.Lists;
 import com.metamx.common.lifecycle.Lifecycle;
 import com.metamx.http.client.HttpClient;
 import com.metamx.http.client.HttpClientConfig;
 import com.metamx.http.client.HttpClientInit;
+import com.metamx.http.client.Request;
 
 import io.druid.query.BaseQuery;
 import io.druid.query.Druids;
 import io.druid.query.Druids.SegmentMetadataQueryBuilder;
 import io.druid.query.Druids.SelectQueryBuilder;
 import io.druid.query.Druids.TimeBoundaryQueryBuilder;
+import io.druid.query.LocatedSegmentDescriptor;
 import io.druid.query.Query;
 import io.druid.query.Result;
+import io.druid.query.SegmentDescriptor;
 import io.druid.query.metadata.metadata.SegmentAnalysis;
 import io.druid.query.metadata.metadata.SegmentMetadataQuery;
 import io.druid.query.select.PagingSpec;
 import io.druid.query.select.SelectQuery;
 import io.druid.query.spec.MultipleIntervalSegmentSpec;
+import io.druid.query.spec.MultipleSpecificSegmentSpec;
 import io.druid.query.timeboundary.TimeBoundaryQuery;
 import io.druid.query.timeboundary.TimeBoundaryResultValue;
 
@@ -143,12 +150,17 @@ public class DruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidW
       case Query.TIMESERIES:
       case Query.TOPN:
       case Query.GROUP_BY:
-        return new HiveDruidSplit[] { new HiveDruidSplit(address,
-                deserializeSerialize(druidQuery), paths[0]) };
+        return new HiveDruidSplit[] { new HiveDruidSplit(deserializeSerialize(druidQuery),
+                paths[0], new String[] {address}) };
       case Query.SELECT:
         SelectQuery selectQuery = DruidStorageHandlerUtils.JSON_MAPPER.readValue(
                 druidQuery, SelectQuery.class);
-        return splitSelectQuery(conf, address, selectQuery, paths[0]);
+        boolean distributed = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_DRUID_SELECT_DISTRIBUTE);
+        if (distributed) {
+          return distributeSelectQuery(conf, address, selectQuery, paths[0]);
+        } else {
+          return splitSelectQuery(conf, address, selectQuery, paths[0]);
+        }
       default:
         throw new IOException("Druid query type not recognized");
     }
@@ -166,8 +178,83 @@ public class DruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidW
     return DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(builder.build());
   }
 
+  /* New method that distributes the Select query by creating splits containing
+   * information about different Druid nodes that have the data for the given
+   * query. */
+  private static HiveDruidSplit[] distributeSelectQuery(Configuration conf, String address,
+      SelectQuery query, Path dummyPath) throws IOException {
+    // If it has a limit, we use it and we do not distribute the query
+    final boolean isFetch = query.getContextBoolean(Constants.DRUID_QUERY_FETCH, false);
+    if (isFetch) {
+      return new HiveDruidSplit[] { new HiveDruidSplit(
+              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath,
+              new String[]{address} ) };
+    }
+
+    // Properties from configuration
+    final int numConnection = HiveConf.getIntVar(conf,
+            HiveConf.ConfVars.HIVE_DRUID_NUM_HTTP_CONNECTION);
+    final Period readTimeout = new Period(
+            HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_DRUID_HTTP_READ_TIMEOUT));
+
+    // Create request to obtain nodes that are holding data for the given datasource and intervals
+    final Lifecycle lifecycle = new Lifecycle();
+    final HttpClient client = HttpClientInit.createClient(
+            HttpClientConfig.builder().withNumConnections(numConnection)
+                    .withReadTimeout(readTimeout.toStandardDuration()).build(), lifecycle);
+    try {
+      lifecycle.start();
+    } catch (Exception e) {
+      LOG.error("Lifecycle start issue");
+      throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+    }
+    final String intervals =
+            StringUtils.join(query.getIntervals(), ","); // Comma-separated intervals without brackets
+    final String request = String.format(
+            "http://%s/druid/v2/datasources/%s/candidates?intervals=%s",
+            address, query.getDataSource().getNames().get(0), intervals);
+    final InputStream response;
+    try {
+      response = DruidStorageHandlerUtils.submitRequest(client, new Request(HttpMethod.GET, new URL(request)));
+    } catch (Exception e) {
+      lifecycle.stop();
+      throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+    }
+
+    // Retrieve results
+    final List<LocatedSegmentDescriptor> segmentDescriptors;
+    try {
+      segmentDescriptors = DruidStorageHandlerUtils.JSON_MAPPER.readValue(response,
+              new TypeReference<List<LocatedSegmentDescriptor>>() {});
+    } catch (Exception e) {
+      response.close();
+      throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+    } finally {
+      lifecycle.stop();
+    }
+
+    // Create one input split for each segment
+    final int numSplits = segmentDescriptors.size();
+    final HiveDruidSplit[] splits = new HiveDruidSplit[segmentDescriptors.size()];
+    for (int i = 0; i < numSplits; i++) {
+      final LocatedSegmentDescriptor locatedSD = segmentDescriptors.get(i);
+      final String[] hosts = new String[locatedSD.getLocations().size()];
+      for (int j = 0; j < locatedSD.getLocations().size(); j++) {
+        hosts[j] = locatedSD.getLocations().get(j).getHost();
+      }
+      // Create partial Select query
+      final SegmentDescriptor newSD = new SegmentDescriptor(
+              locatedSD.getInterval(), locatedSD.getVersion(), locatedSD.getPartitionNumber());
+      final SelectQuery partialQuery = query.withQuerySegmentSpec(
+              new MultipleSpecificSegmentSpec(Lists.newArrayList(newSD)));
+      splits[i] = new HiveDruidSplit(DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery),
+              dummyPath, hosts);
+    }
+    return splits;
+  }
+
   /* Method that splits Select query depending on the threshold so read can be
-   * parallelized */
+   * parallelized. We will only contact the Druid broker to obtain all results. */
   private static HiveDruidSplit[] splitSelectQuery(Configuration conf, String address,
           SelectQuery query, Path dummyPath
   ) throws IOException {
@@ -182,7 +269,8 @@ public class DruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidW
     if (isFetch) {
       // If it has a limit, we use it and we do not split the query
       return new HiveDruidSplit[] { new HiveDruidSplit(
-              address, DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath) };
+              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath,
+              new String[] {address} ) };
     }
 
     // We do not have the number of rows, thus we need to execute a
@@ -200,7 +288,8 @@ public class DruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidW
     try {
       lifecycle.start();
     } catch (Exception e) {
-      LOG.error("Lifecycle start issue", e);
+      LOG.error("Lifecycle start issue");
+      throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
     }
     InputStream response;
     try {
@@ -231,7 +320,8 @@ public class DruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidW
     if (metadataList.isEmpty()) {
       // There are no rows for that time range, we can submit query as it is
       return new HiveDruidSplit[] { new HiveDruidSplit(
-              address, DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath) };
+              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath,
+              new String[] {address} ) };
     }
     if (metadataList.size() != 1) {
       throw new IOException("Information about segments should have been merged");
@@ -242,9 +332,9 @@ public class DruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidW
     query = query.withPagingSpec(PagingSpec.newSpec(Integer.MAX_VALUE));
     if (numRows <= selectThreshold) {
       // We are not going to split it
-      return new HiveDruidSplit[] { new HiveDruidSplit(address,
-              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath
-      ) };
+      return new HiveDruidSplit[] { new HiveDruidSplit(
+              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath,
+              new String[] {address} ) };
     }
 
     // If the query does not specify a timestamp, we obtain the total time using
@@ -266,12 +356,8 @@ public class DruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidW
       try {
         lifecycle.start();
       } catch (Exception e) {
-        LOG.error("Lifecycle start issue", e);
-      }
-      try {
-        lifecycle.start();
-      } catch (Exception e) {
-        LOG.error("Lifecycle start issue", e);
+        LOG.error("Lifecycle start issue");
+        throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
       }
       try {
         response = DruidStorageHandlerUtils.submitRequest(client,
@@ -318,9 +404,9 @@ public class DruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidW
       // Create partial Select query
       final SelectQuery partialQuery = query.withQuerySegmentSpec(
               new MultipleIntervalSegmentSpec(newIntervals.get(i)));
-      splits[i] = new HiveDruidSplit(address,
-              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), dummyPath
-      );
+      splits[i] = new HiveDruidSplit(
+              DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), dummyPath,
+              new String[] {address});
     }
     return splits;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/8ab1889d/druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java
index 861075d..58cb47a 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.druid.io;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileSplit;
@@ -29,56 +30,41 @@ import org.apache.hadoop.mapred.FileSplit;
  */
 public class HiveDruidSplit extends FileSplit implements org.apache.hadoop.mapred.InputSplit {
 
-  private String address;
-
   private String druidQuery;
 
+  private String[] hosts;
+
   // required for deserialization
   public HiveDruidSplit() {
     super((Path) null, 0, 0, (String[]) null);
   }
 
-  public HiveDruidSplit(String address, String druidQuery, Path dummyPath) {
-    super(dummyPath, 0, 0, (String[]) null);
-    this.address = address;
+  public HiveDruidSplit(String druidQuery, Path dummyPath, String hosts[]) {
+    super(dummyPath, 0, 0, hosts);
     this.druidQuery = druidQuery;
+    this.hosts = hosts;
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
-    out.writeUTF(address);
     out.writeUTF(druidQuery);
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
-    address = in.readUTF();
     druidQuery = in.readUTF();
   }
 
-  @Override
-  public long getLength() {
-    return 0L;
-  }
-
-  @Override
-  public String[] getLocations() {
-    return new String[] { "" };
-  }
-
-  public String getAddress() {
-    return address;
-  }
-
   public String getDruidQuery() {
     return druidQuery;
   }
 
   @Override
   public String toString() {
-    return "HiveDruidSplit{" + address + ", " + druidQuery + "}";
+    return "HiveDruidSplit{" + druidQuery + ", " 
+            + (hosts == null ? "empty hosts" : Arrays.toString(hosts))  + "}";
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/8ab1889d/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
index 0d5f0b1..8d099c7 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
@@ -98,8 +98,7 @@ public abstract class DruidQueryRecordReader<T extends BaseQuery<R>, R extends C
     InputStream response;
     try {
       response = DruidStorageHandlerUtils.submitRequest(client,
-              DruidStorageHandlerUtils.createRequest(hiveDruidSplit.getAddress(), query)
-      );
+              DruidStorageHandlerUtils.createRequest(hiveDruidSplit.getLocations()[0], query));
     } catch (Exception e) {
       lifecycle.stop();
       throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));

http://git-wip-us.apache.org/repos/asf/hive/blob/8ab1889d/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
index 9b7a1da..bb4011b 100644
--- a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
@@ -143,8 +143,7 @@ public class TestHiveDruidQueryBasedInputFormat extends TestCase {
           + " \"descending\": \"true\", "
           + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ]}";
   private static final String TIMESERIES_QUERY_SPLIT =
-      "[HiveDruidSplit{localhost:8082, "
-          + "{\"queryType\":\"timeseries\","
+      "[HiveDruidSplit{{\"queryType\":\"timeseries\","
           + "\"dataSource\":{\"type\":\"table\",\"name\":\"sample_datasource\"},"
           + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2012-01-01T00:00:00.000-08:00/2012-01-03T00:00:00.000-08:00\"]},"
           + "\"descending\":true,"
@@ -152,7 +151,7 @@ public class TestHiveDruidQueryBasedInputFormat extends TestCase {
           + "\"granularity\":{\"type\":\"duration\",\"duration\":86400000,\"origin\":\"1969-12-31T16:00:00.000-08:00\"},"
           + "\"aggregations\":[],"
           + "\"postAggregations\":[],"
-          + "\"context\":null}}]";
+          + "\"context\":null}, [localhost:8082]}]";
 
   private static final String TOPN_QUERY =
       "{  \"queryType\": \"topN\", "
@@ -177,8 +176,7 @@ public class TestHiveDruidQueryBasedInputFormat extends TestCase {
           + "  \"2013-08-31T00:00:00.000/2013-09-03T00:00:00.000\" "
           + " ]}";
   private static final String TOPN_QUERY_SPLIT =
-      "[HiveDruidSplit{localhost:8082, "
-          + "{\"queryType\":\"topN\","
+      "[HiveDruidSplit{{\"queryType\":\"topN\","
           + "\"dataSource\":{\"type\":\"table\",\"name\":\"sample_data\"},"
           + "\"dimension\":{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"sample_dim\",\"outputName\":\"sample_dim\"},"
           + "\"metric\":{\"type\":\"LegacyTopNMetricSpec\",\"metric\":\"count\"},"
@@ -190,7 +188,7 @@ public class TestHiveDruidQueryBasedInputFormat extends TestCase {
           + "{\"type\":\"doubleSum\",\"name\":\"some_metric\",\"fieldName\":\"some_metric\"}],"
           + "\"postAggregations\":[],"
           + "\"context\":null,"
-          + "\"descending\":false}}]";
+          + "\"descending\":false}, [localhost:8082]}]";
 
   private static final String GROUP_BY_QUERY =
       "{  \"queryType\": \"groupBy\", "
@@ -208,8 +206,7 @@ public class TestHiveDruidQueryBasedInputFormat extends TestCase {
           + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ]"
           + " }";
   private static final String GROUP_BY_QUERY_SPLIT =
-      "[HiveDruidSplit{localhost:8082, "
-          + "{\"queryType\":\"groupBy\","
+      "[HiveDruidSplit{{\"queryType\":\"groupBy\","
           + "\"dataSource\":{\"type\":\"table\",\"name\":\"sample_datasource\"},"
           + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2012-01-01T00:00:00.000-08:00/2012-01-03T00:00:00.000-08:00\"]},"
           + "\"filter\":null,"
@@ -223,7 +220,7 @@ public class TestHiveDruidQueryBasedInputFormat extends TestCase {
           + "\"limitSpec\":{\"type\":\"default\",\"columns\":[{\"dimension\":\"country\",\"direction\":\"ascending\",\"dimensionOrder\":{\"type\":\"lexicographic\"}},"
           + "{\"dimension\":\"data_transfer\",\"direction\":\"ascending\",\"dimensionOrder\":{\"type\":\"lexicographic\"}}],\"limit\":5000},"
           + "\"context\":null,"
-          + "\"descending\":false}}]";
+          + "\"descending\":false}, [localhost:8082]}]";
 
   private static final String SELECT_QUERY =
       "{   \"queryType\": \"select\",  "
@@ -235,8 +232,7 @@ public class TestHiveDruidQueryBasedInputFormat extends TestCase {
           + " \"pagingSpec\":{\"pagingIdentifiers\": {}, \"threshold\":5}, "
           + " \"context\":{\"druid.query.fetch\":true}}";
   private static final String SELECT_QUERY_SPLIT =
-      "[HiveDruidSplit{localhost:8082, "
-          + "{\"queryType\":\"select\","
+      "[HiveDruidSplit{{\"queryType\":\"select\","
           + "\"dataSource\":{\"type\":\"table\",\"name\":\"wikipedia\"},"
           + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2013-01-01T00:00:00.000-08:00/2013-01-02T00:00:00.000-08:00\"]},"
           + "\"descending\":false,"
@@ -252,7 +248,7 @@ public class TestHiveDruidQueryBasedInputFormat extends TestCase {
           + "{\"type\":\"LegacyDimensionSpec\",\"dimension\":\"user\",\"outputName\":\"user\"}],"
           + "\"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"],"
           + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":5,\"fromNext\":false},"
-          + "\"context\":{\"druid.query.fetch\":true}}}]";
+          + "\"context\":{\"druid.query.fetch\":true}}, [localhost:8082]}]";
 
   @Test
   public void testTimeZone() throws Exception {
@@ -289,6 +285,7 @@ public class TestHiveDruidQueryBasedInputFormat extends TestCase {
     conf.set(Constants.DRUID_DATA_SOURCE, dataSource);
     conf.set(Constants.DRUID_QUERY_JSON, jsonQuery);
     conf.set(Constants.DRUID_QUERY_TYPE, queryType);
+    conf.setBoolean(HiveConf.ConfVars.HIVE_DRUID_SELECT_DISTRIBUTE.varname, false);
     return conf;
   }
 


[10/50] [abbrv] hive git commit: HIVE-15957 : Follow Hive's rules for type inference instead of Calcite (Ashutosh Chauhan via Jesus Camacho Rodriguez)

Posted by se...@apache.org.
HIVE-15957 : Follow Hive's rules for type inference instead of Calcite (Ashutosh Chauhan via Jesus Camacho Rodriguez)

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


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

Branch: refs/heads/hive-14535
Commit: 6f6a5586ef4a05903e81e865914cd3ab9b2e0555
Parents: 1677ed9
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Thu Feb 16 17:05:08 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Feb 17 10:51:59 2017 -0800

----------------------------------------------------------------------
 .../translator/SqlFunctionConverter.java        | 33 +++++++++++-------
 .../clientpositive/interval_arithmetic.q        |  3 ++
 .../results/clientpositive/interval_alt.q.out   |  6 ++--
 .../clientpositive/interval_arithmetic.q.out    | 35 ++++++++++++++++++++
 .../llap/metadata_only_queries.q.out            |  4 +--
 .../clientpositive/metadata_only_queries.q.out  |  4 +--
 .../spark/metadata_only_queries.q.out           |  4 +--
 7 files changed, 68 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6f6a5586/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index 5c85dce..85450c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -329,6 +329,7 @@ public class SqlFunctionConverter {
 
     StaticBlockBuilder() {
       registerFunction("+", SqlStdOperatorTable.PLUS, hToken(HiveParser.PLUS, "+"));
+      registerFunction("-", SqlStdOperatorTable.MINUS, hToken(HiveParser.MINUS, "-"));
       registerFunction("*", SqlStdOperatorTable.MULTIPLY, hToken(HiveParser.STAR, "*"));
       registerFunction("/", SqlStdOperatorTable.DIVIDE, hToken(HiveParser.DIVIDE, "/"));
       registerFunction("%", SqlStdOperatorTable.MOD, hToken(HiveParser.Identifier, "%"));
@@ -482,21 +483,29 @@ public class SqlFunctionConverter {
       // this.So, bail out for now.
       throw new CalciteSemanticException("<=> is not yet supported for cbo.", UnsupportedFeature.Less_than_equal_greater_than);
     }
-    SqlOperator calciteOp = hiveToCalcite.get(hiveUdfName);
-    if (calciteOp == null) {
-      CalciteUDFInfo uInf = getUDFInfo(hiveUdfName, calciteArgTypes, calciteRetType);
-      if ("-".equals(hiveUdfName)) {
-        // Calcite native - has broken inference for return type, so we override it with explicit return type
-        // e.g. timestamp - timestamp is inferred as timestamp, where it really should be interval.
+    SqlOperator calciteOp;
+    CalciteUDFInfo uInf = getUDFInfo(hiveUdfName, calciteArgTypes, calciteRetType);
+    switch (hiveUdfName) {
+      // Follow hive's rules for type inference as oppose to Calcite's
+      // for return type.
+      //TODO: Perhaps we should do this for all functions, not just +,-
+      case "-":
         calciteOp = new SqlMonotonicBinaryOperator("-", SqlKind.MINUS, 40, true,
             uInf.returnTypeInference, uInf.operandTypeInference, OperandTypes.MINUS_OPERATOR);
-      } else {
-        calciteOp = new CalciteSqlFn(uInf.udfName, SqlKind.OTHER_FUNCTION, uInf.returnTypeInference,
-            uInf.operandTypeInference, uInf.operandTypeChecker,
-            SqlFunctionCategory.USER_DEFINED_FUNCTION, deterministic);
-      }
+        break;
+      case "+":
+        calciteOp = new SqlMonotonicBinaryOperator("+", SqlKind.PLUS, 40, true,
+            uInf.returnTypeInference, uInf.operandTypeInference, OperandTypes.PLUS_OPERATOR);
+        break;
+      default:
+        calciteOp = hiveToCalcite.get(hiveUdfName);
+        if (null == calciteOp) {
+          calciteOp = new CalciteSqlFn(uInf.udfName, SqlKind.OTHER_FUNCTION, uInf.returnTypeInference,
+              uInf.operandTypeInference, uInf.operandTypeChecker,
+              SqlFunctionCategory.USER_DEFINED_FUNCTION, deterministic);
+        }
+        break;
     }
-
     return calciteOp;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/6f6a5586/ql/src/test/queries/clientpositive/interval_arithmetic.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/interval_arithmetic.q b/ql/src/test/queries/clientpositive/interval_arithmetic.q
index 06acbd7..445cdfe 100644
--- a/ql/src/test/queries/clientpositive/interval_arithmetic.q
+++ b/ql/src/test/queries/clientpositive/interval_arithmetic.q
@@ -159,4 +159,7 @@ select
 from interval_arithmetic_1
 limit 2;
 
+explain
+select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1;
+select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1;
 drop table interval_arithmetic_1;

http://git-wip-us.apache.org/repos/asf/hive/blob/6f6a5586/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
index 9884ec3..eba8420 100644
--- a/ql/src/test/results/clientpositive/interval_alt.q.out
+++ b/ql/src/test/results/clientpositive/interval_alt.q.out
@@ -137,7 +137,7 @@ STAGE PLANS:
             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)
+              expressions: (2012-01-01 + IntervalDayLiteralProcessor(((- dt) * dt))) (type: timestamp), (2012-01-01 - IntervalDayLiteralProcessor(((- dt) * dt))) (type: timestamp), 2012-01-04 00:00:00.0 (type: timestamp), (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
@@ -172,5 +172,5 @@ POSTHOOK: query: select
 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
+2011-12-31 00:00:00	2012-01-02 00:00:00	2012-01-04 00:00:00	2013-02-01
+2011-12-28 00:00:00	2012-01-05 00:00:00	2012-01-04 00:00:00	2014-02-01

http://git-wip-us.apache.org/repos/asf/hive/blob/6f6a5586/ql/src/test/results/clientpositive/interval_arithmetic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/interval_arithmetic.q.out b/ql/src/test/results/clientpositive/interval_arithmetic.q.out
index 64882f8..c1fc738 100644
--- a/ql/src/test/results/clientpositive/interval_arithmetic.q.out
+++ b/ql/src/test/results/clientpositive/interval_arithmetic.q.out
@@ -606,6 +606,41 @@ POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
 109 20:30:40.246913578	89 02:14:26.000000000
 109 20:30:40.246913578	89 02:14:26.000000000
+PREHOOK: query: explain
+select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        TableScan
+          alias: interval_arithmetic_1
+          Statistics: Num rows: 12288 Data size: 326837 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: 2017-02-19 03:04:00.0 (type: timestamp)
+            outputColumnNames: _col0
+            Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
+            Limit
+              Number of rows: 1
+              Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+              ListSink
+
+PREHOOK: query: select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@interval_arithmetic_1
+#### A masked pattern was here ####
+POSTHOOK: query: select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@interval_arithmetic_1
+#### A masked pattern was here ####
+2017-02-19 03:04:00
 PREHOOK: query: drop table interval_arithmetic_1
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@interval_arithmetic_1

http://git-wip-us.apache.org/repos/asf/hive/blob/6f6a5586/ql/src/test/results/clientpositive/llap/metadata_only_queries.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/metadata_only_queries.q.out b/ql/src/test/results/clientpositive/llap/metadata_only_queries.q.out
index 25be543..c8190bd 100644
--- a/ql/src/test/results/clientpositive/llap/metadata_only_queries.q.out
+++ b/ql/src/test/results/clientpositive/llap/metadata_only_queries.q.out
@@ -343,7 +343,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                 Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(11,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
+                  expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(2,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                   Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
@@ -407,7 +407,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                 Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(11,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
+                  expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(2,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                   Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/6f6a5586/ql/src/test/results/clientpositive/metadata_only_queries.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/metadata_only_queries.q.out b/ql/src/test/results/clientpositive/metadata_only_queries.q.out
index 2e3331e..57b59dd 100644
--- a/ql/src/test/results/clientpositive/metadata_only_queries.q.out
+++ b/ql/src/test/results/clientpositive/metadata_only_queries.q.out
@@ -313,7 +313,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
           Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(11,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
+            expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(2,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
             Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
@@ -367,7 +367,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
           Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(11,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
+            expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(2,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
             Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
             File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/6f6a5586/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out b/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out
index dc96a0d..543d0ef 100644
--- a/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out
+++ b/ql/src/test/results/clientpositive/spark/metadata_only_queries.q.out
@@ -331,7 +331,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                 Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(11,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
+                  expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(2,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                   Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
@@ -391,7 +391,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
                 Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(11,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
+                  expressions: _col0 (type: bigint), '1' (type: string), _col1 (type: bigint), _col2 (type: decimal(11,1)), 2 (type: int), _col3 (type: bigint), _col4 (type: bigint), 7 (type: decimal(2,0)), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: bigint), _col8 (type: int), _col9 (type: bigint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                   Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator


[41/50] [abbrv] hive git commit: HIVE-15955: make explain formatted to include opId and etc (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-15955: make explain formatted to include opId and etc (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/759766ee
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/759766ee
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/759766ee

Branch: refs/heads/hive-14535
Commit: 759766eeb5d8047d4acdbd2faca755985bb5a39f
Parents: 89310fe
Author: Pengcheng Xiong <px...@apache.org>
Authored: Wed Feb 22 20:10:19 2017 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Wed Feb 22 20:10:19 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/ExplainTask.java | 10 ++-
 .../AnnotateReduceSinkOutputOperator.java       | 73 ++++++++++++++++++++
 .../hadoop/hive/ql/optimizer/Optimizer.java     |  4 ++
 .../hadoop/hive/ql/plan/ReduceSinkDesc.java     | 14 ++++
 .../clientpositive/explain_formatted_oid.q      | 18 +++++
 .../clientpositive/explain_formatted_oid.q.out  | 38 ++++++++++
 ql/src/test/results/clientpositive/input4.q.out |  2 +-
 ql/src/test/results/clientpositive/join0.q.out  |  2 +-
 .../results/clientpositive/parallel_join0.q.out |  2 +-
 .../test/results/clientpositive/plan_json.q.out |  2 +-
 .../clientpositive/vector_outer_join3.q.out     |  6 +-
 .../clientpositive/vector_outer_join4.q.out     |  6 +-
 .../clientpositive/vector_outer_join6.q.out     |  4 +-
 13 files changed, 168 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
index 74cec3e..086ccb2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
@@ -115,6 +115,7 @@ import org.slf4j.LoggerFactory;
 public class ExplainTask extends Task<ExplainWork> implements Serializable {
   private static final long serialVersionUID = 1L;
   public static final String EXPL_COLUMN_NAME = "Explain";
+  public static final String OUTPUT_OPERATORS = "OutputOperators:";
   private final Set<Operator<?>> visitedOps = new HashSet<Operator<?>>();
   private boolean isLogical = false;
   protected final Logger LOG;
@@ -790,10 +791,17 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
         String appender = isLogical ? " (" + operator.getOperatorId() + ")" : "";
         JSONObject jsonOut = outputPlan(operator.getConf(), out, extended,
             jsonOutput, jsonOutput ? 0 : indent, appender);
-        if (this.work != null && this.work.isUserLevelExplain()) {
+        if (this.work != null && (this.work.isUserLevelExplain() || this.work.isFormatted())) {
           if (jsonOut != null && jsonOut.length() > 0) {
             ((JSONObject) jsonOut.get(JSONObject.getNames(jsonOut)[0])).put("OperatorId:",
                 operator.getOperatorId());
+            if (!this.work.isUserLevelExplain() && this.work.isFormatted()
+                && operator instanceof ReduceSinkOperator) {
+              ((JSONObject) jsonOut.get(JSONObject.getNames(jsonOut)[0])).put(
+                  OUTPUT_OPERATORS,
+                  Arrays.toString(((ReduceSinkOperator) operator).getConf().getOutputOperators()
+                      .toArray()));
+            }
           }
         }
         if (jsonOutput) {

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AnnotateReduceSinkOutputOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AnnotateReduceSinkOutputOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AnnotateReduceSinkOutputOperator.java
new file mode 100644
index 0000000..0b61f4b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AnnotateReduceSinkOutputOperator.java
@@ -0,0 +1,73 @@
+/**
+ * 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.optimizer;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
+import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
+import org.apache.hadoop.hive.ql.lib.GraphWalker;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.RuleRegExp;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+
+/**
+ * Implementation of AnnotateReduceSinkOutputOperator optimization step.
+ */
+public class AnnotateReduceSinkOutputOperator extends Transform {
+  @Override
+  public ParseContext transform(ParseContext pctx) throws SemanticException {
+
+    // 1. We apply the transformation
+    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
+    opRules.put(new RuleRegExp("R1",
+      "(" + ReduceSinkOperator.getOperatorName() + "%)"), new ReduceSinkOutputOperatorAnnotator());
+    GraphWalker ogw = new DefaultGraphWalker(new DefaultRuleDispatcher(null, opRules, null));
+    ArrayList<Node> topNodes = new ArrayList<Node>();
+    topNodes.addAll(pctx.getTopOps().values());
+    ogw.startWalking(topNodes, null);
+    return pctx;
+  }
+
+  private static class ReduceSinkOutputOperatorAnnotator implements NodeProcessor {
+    @Override
+    public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
+        Object... nodeOutputs) throws SemanticException {
+      ReduceSinkOperator rs = (ReduceSinkOperator) nd;
+      List<Operator<? extends OperatorDesc>> children = rs.getChildOperators();
+      List<String> outputOperators = new ArrayList<>();
+      for (Operator<? extends OperatorDesc> operator : children) {
+        outputOperators.add(operator.getOperatorId());
+      }
+      rs.getConf().setOutputOperators(outputOperators);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
index c6287e4..a3a19f4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
@@ -232,6 +232,10 @@ public class Optimizer {
       transformations.add(new SimpleFetchAggregation());
     }
 
+    if (pctx.getContext().getExplainConfig() != null
+        && pctx.getContext().getExplainConfig().isFormatted()) {
+      transformations.add(new AnnotateReduceSinkOutputOperator());
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
index b8c2d42..d77a223 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
@@ -80,6 +80,12 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
   private String outputName;
 
   /**
+   * Holds the name of the output operators
+   * that this reduce sink is outputing to.
+   */
+  private List<String> outputOperators;
+
+  /**
    * The partition columns (CLUSTER BY or DISTRIBUTE BY in Hive language).
    * Partition columns decide the reducer that the current row goes to.
    * Partition columns are not passed to reducer.
@@ -587,4 +593,12 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
     }
     return new ReduceSinkOperatorExplainVectorization(this, vectorDesc);
   }
+
+  public List<String> getOutputOperators() {
+    return outputOperators;
+  }
+
+  public void setOutputOperators(List<String> outputOperators) {
+    this.outputOperators = outputOperators;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/queries/clientpositive/explain_formatted_oid.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/explain_formatted_oid.q b/ql/src/test/queries/clientpositive/explain_formatted_oid.q
new file mode 100644
index 0000000..932f119
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/explain_formatted_oid.q
@@ -0,0 +1,18 @@
+set hive.auto.convert.join=false;
+
+create table srcTable (key string, value string);
+
+explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN srcTable z ON (x.value = z.value);
+
+explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+union
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value);
+

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/explain_formatted_oid.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_formatted_oid.q.out b/ql/src/test/results/clientpositive/explain_formatted_oid.q.out
new file mode 100644
index 0000000..46c2090
--- /dev/null
+++ b/ql/src/test/results/clientpositive/explain_formatted_oid.q.out
@@ -0,0 +1,38 @@
+PREHOOK: query: create table srcTable (key string, value string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcTable
+POSTHOOK: query: create table srcTable (key string, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcTable
+PREHOOK: query: explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN srcTable z ON (x.value = z.value)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN srcTable z ON (x.value = z.value)
+POSTHOOK: type: QUERY
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"x","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_0","children":{"Filter Operator":{"predicate:":"(key is not null and value is not null) (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_17","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_2","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NON
 E","value expressions:":"_col1 (type: string)","OperatorId:":"RS_9","OutputOperators:":"[JOIN_11]"}}}}}}}},{"TableScan":{"alias:":"y","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_3","children":{"Filter Operator":{"predicate:":"key is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_18","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_5","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col1 (type: string)","OperatorId:":"RS_10","OutputOperators:":"[JOIN_11]"}}}}}}}}],"Reduce 
 Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"outputColumnNames:":["_col0","_col1","_col3"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_11","children":{"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"},"OperatorId:":"FS_20"}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_21","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col0 (type: string), _col3 (type: string)","OperatorId
 :":"RS_12","OutputOperators:":"[JOIN_14]"}}}},{"TableScan":{"alias:":"z","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_6","children":{"Filter Operator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_19","children":{"Select Operator":{"expressions:":"value (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_8","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_13","OutputOperators:":"[JOIN_14]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type:
  string)"},"outputColumnNames:":["_col0","_col3","_col4"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_14","children":{"Select Operator":{"expressions:":"_col0 (type: string), _col4 (type: string), _col3 (type: string)","outputColumnNames:":["_col0","_col1","_col2"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL 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"},"OperatorId:":"FS_16"}}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_22"}}}}}}
+PREHOOK: query: explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+union
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain formatted
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+union
+SELECT x.key, z.value, y.value
+FROM srcTable x JOIN srcTable y ON (x.key = y.key) 
+JOIN (select * from srcTable union select * from srcTable)z ON (x.value = z.value)
+POSTHOOK: type: QUERY
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1, Stage-5"},"Stage-3":{"DEPENDENT STAGES":"Stage-2, Stage-8"},"Stage-5":{"ROOT STAGE":"TRUE"},"Stage-7":{"ROOT STAGE":"TRUE"},"Stage-8":{"DEPENDENT STAGES":"Stage-7, Stage-10"},"Stage-10":{"ROOT STAGE":"TRUE"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"x","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_0","children":{"Filter Operator":{"predicate:":"(key is not null and value is not null) (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_60","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_2","children":{"Reduce Outp
 ut Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col1 (type: string)","OperatorId:":"RS_19","OutputOperators:":"[JOIN_21]"}}}}}}}},{"TableScan":{"alias:":"y","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_3","children":{"Filter Operator":{"predicate:":"key is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_61","children":{"Select Operator":{"expressions:":"key (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_5","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Nu
 m rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_20","OutputOperators:":"[JOIN_21]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_21","children":{"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"},"OperatorId:":"FS_68"}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_69","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data 
 size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col0 (type: string)","OperatorId:":"RS_22","OutputOperators:":"[JOIN_24]"}}}},{"TableScan":{"OperatorId:":"TS_73","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_23","OutputOperators:":"[JOIN_24]"}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col1 (type: string)"},"outputColumnNames:":["_col0","_col4"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_24","children":{"Select Operator":{"expressions:":"_col0 (type: string), _col4 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":
 "SEL_25","children":{"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"},"OperatorId:":"FS_70"}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_71","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_52","children":{"Group By Operator":{"keys:":"_col0 (type: string), _col1 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_55","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 
 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_56","OutputOperators:":"[GBY_57]"}}}}}}}},{"TableScan":{"OperatorId:":"TS_77","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_52","children":{"Group By Operator":{"keys:":"_col0 (type: string), _col1 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_55","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_56","OutputOperators:":"[GBY_57]"}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"keys:":"KEY._col0 (type: string), KEY._col1 (type: string)","mode:":"mergepartial","out
 putColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_57","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL 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"},"OperatorId:":"FS_59"}}}}}},"Stage-5":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"srctable","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_6","children":{"Filter Operator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_62","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames
 :":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_8","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_12","children":{"Group By Operator":{"keys:":"_col1 (type: string), _col0 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_15","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_16","OutputOperators:":"[GBY_17]"}}}}}}}}}}}},{"TableScan":{"alias:":"srctable","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_9","children":{"Filter Ope
 rator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_63","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_11","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_12","children":{"Group By Operator":{"keys:":"_col1 (type: string), _col0 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_15","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data siz
 e: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_16","OutputOperators:":"[GBY_17]"}}}}}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"keys:":"KEY._col0 (type: string), KEY._col1 (type: string)","mode:":"mergepartial","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_17","children":{"Select Operator":{"expressions:":"_col0 (type: string)","outputColumnNames:":["_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_18","children":{"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"},"OperatorId:":"FS_72"}}}}}}}},"Stage-7":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"x","Statistics:":"Num rows:
  1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_26","children":{"Filter Operator":{"predicate:":"(key is not null and value is not null) (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_64","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_28","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col1 (type: string)","OperatorId:":"RS_45","OutputOperators:":"[JOIN_47]"}}}}}}}},{"TableScan":{"alias:":"y","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_29","children":
 {"Filter Operator":{"predicate:":"key is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_65","children":{"Select Operator":{"expressions:":"key (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_31","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col0 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_46","OutputOperators:":"[JOIN_47]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_47","children":{"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"},"OperatorId:":"FS_74"}}}}}},"Stage-8":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_75","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","value expressions:":"_col0 (type: string)","OperatorId:":"RS_48","OutputOperators:":"[JOIN_50]"}}}},{"TableScan":{"OperatorId:":"TS_79","children":{"Reduce Output Operator":{"key expressions:":"_col1 (type: string)","sort order:":"+","Map-reduce partition columns:":"_col1 (type: string)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_49","OutputOpera
 tors:":"[JOIN_50]"}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col1 (type: string)"},"outputColumnNames:":["_col0","_col4"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"JOIN_50","children":{"Select Operator":{"expressions:":"_col0 (type: string), _col4 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_51","children":{"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"},"OperatorId:":"FS_76"}}}}}}}},"Stage-10":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"srctable","Statistics:":"Num rows: 1 Data size: 0 Basic stats:
  PARTIAL Column stats: NONE","OperatorId:":"TS_32","children":{"Filter Operator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_66","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_34","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_38","children":{"Group By Operator":{"keys:":"_col1 (type: string), _col0 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_41","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0
  (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_42","OutputOperators:":"[GBY_43]"}}}}}}}}}}}},{"TableScan":{"alias:":"srctable","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"TS_35","children":{"Filter Operator":{"predicate:":"value is not null (type: boolean)","Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"FIL_67","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_37","children":{"Union":{"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"UNION_38","children":{"Group By Operator":{"keys:":"_col1 (type: string), _col0 (type: string)","mode:":"hash","outputColumnNames:":["_col0","_
 col1"],"Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_41","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string)","sort order:":"++","Map-reduce partition columns:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"RS_42","OutputOperators:":"[GBY_43]"}}}}}}}}}}}}],"Reduce Operator Tree:":{"Group By Operator":{"keys:":"KEY._col0 (type: string), KEY._col1 (type: string)","mode:":"mergepartial","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"GBY_43","children":{"Select Operator":{"expressions:":"_col0 (type: string)","outputColumnNames:":["_col1"],"Statistics:":"Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE","OperatorId:":"SEL_44","children":{"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"},"OperatorId:":"FS_78"}}}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_80"}}}}}}

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/input4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/input4.q.out b/ql/src/test/results/clientpositive/input4.q.out
index 83912f6..6984318 100644
--- a/ql/src/test/results/clientpositive/input4.q.out
+++ b/ql/src/test/results/clientpositive/input4.q.out
@@ -44,7 +44,7 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN FORMATTED
 SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"TableScan":{"alias:":"input4alias","Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"value (type: string), key (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","children":{"ListSink":{}}}}}}}}}}
+{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"TableScan":{"alias:":"input4alias","Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"value (type: string), key (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_1","children":{"ListSink":{"OperatorId:":"LIST_SINK_3"}}}}}}}}}}
 PREHOOK: query: SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
 PREHOOK: type: QUERY
 PREHOOK: Input: default@input4

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join0.q.out b/ql/src/test/results/clientpositive/join0.q.out
index c02319e..b1e85a3 100644
--- a/ql/src/test/results/clientpositive/join0.q.out
+++ b/ql/src/test/results/clientpositive/join0.q.out
@@ -112,7 +112,7 @@ SELECT src1.key as k1, src1.value as v1,
   (SELECT * FROM src WHERE src.key < 10) src2
   SORT BY k1, v1, k2, v2
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 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: COM
 PLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 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:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","children":{"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-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 27556 Data size: 612872 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-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_0","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"FIL_13","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_2","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)","OperatorId:":"RS_6","OutputOper
 ators:":"[JOIN_8]"}}}}}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_3","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"FIL_14","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)","OperatorId:":"RS_7","OutputOperators:":"[JOIN_8]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_
 col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"JOIN_8","children":{"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"},"OperatorId:":"FS_15"}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_16","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"RS_10","OutputOperators:":"[SEL_11]"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.
 reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_11","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 27556 Data size: 612872 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"},"OperatorId:":"FS_12"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_17"}}}}}}
 Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/parallel_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parallel_join0.q.out b/ql/src/test/results/clientpositive/parallel_join0.q.out
index c02319e..b1e85a3 100644
--- a/ql/src/test/results/clientpositive/parallel_join0.q.out
+++ b/ql/src/test/results/clientpositive/parallel_join0.q.out
@@ -112,7 +112,7 @@ SELECT src1.key as k1, src1.value as v1,
   (SELECT * FROM src WHERE src.key < 10) src2
   SORT BY k1, v1, k2, v2
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 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: COM
 PLETE Column stats: NONE","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 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:":{},"outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","children":{"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-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 27556 Data size: 612872 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-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_0","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"FIL_13","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_2","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)","OperatorId:":"RS_6","OutputOper
 ators:":"[JOIN_8]"}}}}}}}},{"TableScan":{"alias:":"src","Statistics:":"Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_3","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"FIL_14","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"Reduce Output Operator":{"sort order:":"","Statistics:":"Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: string), _col1 (type: string)","OperatorId:":"RS_7","OutputOperators:":"[JOIN_8]"}}}}}}}}],"Reduce Operator Tree:":{"Join Operator":{"condition map:":[{"":"Inner Join 0 to 1"}],"keys:":{},"outputColumnNames:":["_col0","_col1","_col2","_
 col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"JOIN_8","children":{"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"},"OperatorId:":"FS_15"}}}}}},"Stage-2":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"OperatorId:":"TS_16","children":{"Reduce Output Operator":{"key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","sort order:":"++++","Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"RS_10","OutputOperators:":"[SEL_11]"}}}}],"Reduce Operator Tree:":{"Select Operator":{"expressions:":"KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.
 reducesinkkey3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Statistics:":"Num rows: 27556 Data size: 612872 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_11","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 27556 Data size: 612872 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"},"OperatorId:":"FS_12"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_17"}}}}}}
 Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/plan_json.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/plan_json.q.out b/ql/src/test/results/clientpositive/plan_json.q.out
index 75d5b73..ba6d0be 100644
--- a/ql/src/test/results/clientpositive/plan_json.q.out
+++ b/ql/src/test/results/clientpositive/plan_json.q.out
@@ -2,4 +2,4 @@ PREHOOK: query: EXPLAIN FORMATTED SELECT count(1) FROM src
 PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN FORMATTED SELECT count(1) FROM src
 POSTHOOK: type: QUERY
-{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"1","Processor Tree:":{"ListSink":{}}}}}}
+{"STAGE DEPENDENCIES":{"Stage-0":{"ROOT STAGE":"TRUE"}},"STAGE PLANS":{"Stage-0":{"Fetch Operator":{"limit:":"1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_8"}}}}}}


[44/50] [abbrv] hive git commit: HIVE-16010 : incorrect conf.set in TezSessionPoolManager (Sergey Shelukhin, reviewed by Illya Yalovyy, Siddharth Seth)

Posted by se...@apache.org.
HIVE-16010 : incorrect conf.set in TezSessionPoolManager (Sergey Shelukhin, reviewed by Illya Yalovyy, Siddharth Seth)


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

Branch: refs/heads/hive-14535
Commit: 53f03358377f3dde21f58e6c841142c6db8a9c32
Parents: 6ca79e3
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Feb 23 11:44:44 2017 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Feb 23 11:44:44 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/53f03358/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java
index ecac85c..b70dbd8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionPoolManager.java
@@ -314,7 +314,7 @@ public class TezSessionPoolManager {
         LOG.warn("User has specified " + queueName + " queue; ignoring the setting");
         queueName = null;
         hasQueue = false;
-        conf.set("tez.queue.name", null);
+        conf.unset("tez.queue.name");
       }
       default: // All good.
       }


[47/50] [abbrv] hive git commit: HIVE-16020: LLAP : Reduce IPC connection misses (Rajesh Balamohan, Siddharth Seth, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-16020: LLAP : Reduce IPC connection misses (Rajesh Balamohan, Siddharth Seth, 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/b8d7192f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b8d7192f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b8d7192f

Branch: refs/heads/hive-14535
Commit: b8d7192f5f28dbc832d4de3e4afc763523a4bf12
Parents: 657236e
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Fri Feb 24 04:00:20 2017 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Fri Feb 24 04:00:20 2017 +0530

----------------------------------------------------------------------
 .../hive/llap/daemon/impl/AMReporter.java       |  5 ++--
 .../llap/daemon/impl/ContainerRunnerImpl.java   | 16 ++++++----
 .../hive/llap/daemon/impl/LlapDaemon.java       |  8 +++--
 .../hadoop/hive/llap/daemon/impl/QueryInfo.java | 29 ++++++++++++++++++
 .../hive/llap/daemon/impl/QueryTracker.java     |  4 ++-
 .../llap/daemon/impl/TaskRunnerCallable.java    | 31 +++++++++++---------
 .../daemon/impl/TaskExecutorTestHelpers.java    |  4 ++-
 7 files changed, 72 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b8d7192f/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/AMReporter.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/AMReporter.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/AMReporter.java
index 93237e6..a30f8b9 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/AMReporter.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/AMReporter.java
@@ -119,7 +119,8 @@ public class AMReporter extends AbstractService {
   private final DaemonId daemonId;
 
   public AMReporter(int numExecutors, int maxThreads, AtomicReference<InetSocketAddress>
-      localAddress, QueryFailedHandler queryFailedHandler, Configuration conf, DaemonId daemonId) {
+      localAddress, QueryFailedHandler queryFailedHandler, Configuration conf, DaemonId daemonId,
+      SocketFactory socketFactory) {
     super(AMReporter.class.getName());
     this.localAddress = localAddress;
     this.queryFailedHandler = queryFailedHandler;
@@ -151,7 +152,7 @@ public class AMReporter extends AbstractService {
         .retryUpToMaximumTimeWithFixedSleep(retryTimeout, retrySleep,
             TimeUnit.MILLISECONDS);
 
-    this.socketFactory = NetUtils.getDefaultSocketFactory(conf);
+    this.socketFactory = socketFactory;
 
     LOG.info("Setting up AMReporter with " +
         "heartbeatInterval(ms)=" + heartbeatInterval +

http://git-wip-us.apache.org/repos/asf/hive/blob/b8d7192f/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
index 6908138..cc4eff0 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/ContainerRunnerImpl.java
@@ -83,6 +83,8 @@ import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
 
+import javax.net.SocketFactory;
+
 public class ContainerRunnerImpl extends CompositeService implements ContainerRunner, FragmentCompletionHandler, QueryFailedHandler {
 
   // TODO Setup a set of threads to process incoming requests.
@@ -107,12 +109,14 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
   private final String clusterId;
   private final DaemonId daemonId;
   private final UgiFactory fsUgiFactory;
+  private final SocketFactory socketFactory;
 
   public ContainerRunnerImpl(Configuration conf, int numExecutors, int waitQueueSize,
       boolean enablePreemption, String[] localDirsBase, AtomicReference<Integer> localShufflePort,
       AtomicReference<InetSocketAddress> localAddress,
       long totalMemoryAvailableBytes, LlapDaemonExecutorMetrics metrics,
-      AMReporter amReporter, ClassLoader classLoader, DaemonId daemonId, UgiFactory fsUgiFactory) {
+      AMReporter amReporter, ClassLoader classLoader, DaemonId daemonId, UgiFactory fsUgiFactory,
+      SocketFactory socketFactory) {
     super("ContainerRunnerImpl");
     Preconditions.checkState(numExecutors > 0,
         "Invalid number of executors: " + numExecutors + ". Must be > 0");
@@ -122,6 +126,7 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
     this.signer = UserGroupInformation.isSecurityEnabled()
         ? new LlapSignerImpl(conf, daemonId.getClusterString()) : null;
     this.fsUgiFactory = fsUgiFactory;
+    this.socketFactory = socketFactory;
 
     this.clusterId = daemonId.getClusterString();
     this.daemonId = daemonId;
@@ -239,7 +244,8 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
           queryIdentifier, qIdProto.getApplicationIdString(), dagId,
           vertex.getDagName(), vertex.getHiveQueryId(), dagIdentifier,
           vertex.getVertexName(), request.getFragmentNumber(), request.getAttemptNumber(),
-          vertex.getUser(), vertex, jobToken, fragmentIdString, tokenInfo);
+          vertex.getUser(), vertex, jobToken, fragmentIdString, tokenInfo, request.getAmHost(),
+          request.getAmPort());
 
       String[] localDirs = fragmentInfo.getLocalDirs();
       Preconditions.checkNotNull(localDirs);
@@ -250,12 +256,12 @@ public class ContainerRunnerImpl extends CompositeService implements ContainerRu
       // Used for re-localization, to add the user specified configuration (conf_pb_binary_stream)
 
       Configuration callableConf = new Configuration(getConfig());
-      UserGroupInformation taskUgi = fsUgiFactory == null ? null : fsUgiFactory.createUgi();
+      UserGroupInformation fsTaskUgi = fsUgiFactory == null ? null : fsUgiFactory.createUgi();
       TaskRunnerCallable callable = new TaskRunnerCallable(request, fragmentInfo, callableConf,
           new ExecutionContextImpl(localAddress.get().getHostName()), env,
           credentials, memoryPerExecutor, amReporter, confParams, metrics, killedTaskHandler,
-          this, tezHadoopShim, attemptId, vertex, initialEvent, taskUgi,
-          completionListener);
+          this, tezHadoopShim, attemptId, vertex, initialEvent, fsTaskUgi,
+          completionListener, socketFactory);
       submissionState = executorService.schedule(callable);
 
       if (LOG.isInfoEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b8d7192f/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index fc9f530..eb05f4c 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
 import javax.management.ObjectName;
+import javax.net.SocketFactory;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.JvmPauseMonitor;
@@ -64,6 +65,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge.UdfWhitelistChecker;
 import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ExitUtil;
@@ -105,6 +107,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
   private final long maxJvmMemory;
   private final String[] localDirs;
   private final DaemonId daemonId;
+  private final SocketFactory socketFactory;
 
   // TODO Not the best way to share the address
   private final AtomicReference<InetSocketAddress> srvAddress = new AtomicReference<>(),
@@ -255,8 +258,9 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
         " sessionId: " + sessionId);
 
     int maxAmReporterThreads = HiveConf.getIntVar(daemonConf, ConfVars.LLAP_DAEMON_AM_REPORTER_MAX_THREADS);
+    this.socketFactory = NetUtils.getDefaultSocketFactory(daemonConf);
     this.amReporter = new AMReporter(numExecutors, maxAmReporterThreads, srvAddress,
-        new QueryFailedHandlerProxy(), daemonConf, daemonId);
+        new QueryFailedHandlerProxy(), daemonConf, daemonId, socketFactory);
 
     SecretManager sm = null;
     if (UserGroupInformation.isSecurityEnabled()) {
@@ -274,7 +278,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     }
     this.containerRunner = new ContainerRunnerImpl(daemonConf, numExecutors, waitQueueSize,
         enablePreemption, localDirs, this.shufflePort, srvAddress, executorMemoryPerInstance, metrics,
-        amReporter, executorClassLoader, daemonId, fsUgiFactory);
+        amReporter, executorClassLoader, daemonId, fsUgiFactory, socketFactory);
     addIfService(containerRunner);
 
     // Not adding the registry as a service, since we need to control when it is initialized - conf used to pickup properties.

http://git-wip-us.apache.org/repos/asf/hive/blob/b8d7192f/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
index 1080d3e..eaa3e7e 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryInfo.java
@@ -16,6 +16,7 @@ package org.apache.hadoop.hive.llap.daemon.impl;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -25,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantLock;
 
 import com.google.common.base.Preconditions;
@@ -36,6 +38,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.llap.daemon.FinishableStateUpdateHandler;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SignableVertexSpec;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.SourceStateProto;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.tez.common.security.JobTokenIdentifier;
 
 public class QueryInfo {
   private final QueryIdentifier queryIdentifier;
@@ -57,6 +64,7 @@ public class QueryInfo {
 
   private final FinishableStateTracker finishableStateTracker = new FinishableStateTracker();
   private final String tokenUserName, appId;
+  private final AtomicReference<UserGroupInformation> umbilicalUgi;
 
   public QueryInfo(QueryIdentifier queryIdentifier, String appIdString, String dagIdString,
                    String dagName, String hiveQueryIdString,
@@ -76,6 +84,7 @@ public class QueryInfo {
     this.localFs = localFs;
     this.tokenUserName = tokenUserName;
     this.appId = tokenAppId;
+    this.umbilicalUgi = new AtomicReference<>();
   }
 
   public QueryIdentifier getQueryIdentifier() {
@@ -297,4 +306,24 @@ public class QueryInfo {
   public String getTokenAppId() {
     return appId;
   }
+
+  public void setupUmbilicalUgi(String umbilicalUser, Token<JobTokenIdentifier> appToken, String amHost, int amPort) {
+    synchronized (umbilicalUgi) {
+      if (umbilicalUgi.get() == null) {
+        UserGroupInformation taskOwner =
+            UserGroupInformation.createRemoteUser(umbilicalUser);
+        final InetSocketAddress address =
+            NetUtils.createSocketAddrForHost(amHost, amPort);
+        SecurityUtil.setTokenService(appToken, address);
+        taskOwner.addToken(appToken);
+        umbilicalUgi.set(taskOwner);
+      }
+    }
+  }
+
+  public UserGroupInformation getUmbilicalUgi() {
+    synchronized (umbilicalUgi) {
+      return umbilicalUgi.get();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b8d7192f/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
index 9eaddd2..5cf3a38 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/QueryTracker.java
@@ -139,7 +139,7 @@ public class QueryTracker extends AbstractService {
   QueryFragmentInfo registerFragment(QueryIdentifier queryIdentifier, String appIdString, String dagIdString,
       String dagName, String hiveQueryIdString, int dagIdentifier, String vertexName, int fragmentNumber, int attemptNumber,
       String user, SignableVertexSpec vertex, Token<JobTokenIdentifier> appToken,
-      String fragmentIdString, LlapTokenInfo tokenInfo) throws IOException {
+      String fragmentIdString, LlapTokenInfo tokenInfo, String amHost, int amPort) throws IOException {
 
     ReadWriteLock dagLock = getDagLock(queryIdentifier);
     // Note: This is a readLock to prevent a race with queryComplete. Operations
@@ -174,6 +174,8 @@ public class QueryTracker extends AbstractService {
         if (old != null) {
           queryInfo = old;
         } else {
+          // Ensure the UGI is setup once.
+          queryInfo.setupUmbilicalUgi(vertex.getTokenIdentifier(), appToken, amHost, amPort);
           isExistingQueryInfo = false;
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/b8d7192f/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
index 4b677aa..8fce546 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.hive.ql.io.IOContextMap;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.log4j.MDC;
@@ -65,6 +64,7 @@ import org.apache.tez.runtime.task.TezTaskRunner2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.net.SocketFactory;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
@@ -116,7 +116,8 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
   private final SignableVertexSpec vertex;
   private final TezEvent initialEvent;
   private final SchedulerFragmentCompletingListener completionListener;
-  private UserGroupInformation taskUgi;
+  private UserGroupInformation fsTaskUgi;
+  private final SocketFactory socketFactory;
 
   @VisibleForTesting
   public TaskRunnerCallable(SubmitWorkRequestProto request, QueryFragmentInfo fragmentInfo,
@@ -125,7 +126,8 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
                             LlapDaemonExecutorMetrics metrics, KilledTaskHandler killedTaskHandler,
                             FragmentCompletionHandler fragmentCompleteHandler, HadoopShim tezHadoopShim,
                             TezTaskAttemptID attemptId, SignableVertexSpec vertex, TezEvent initialEvent,
-                            UserGroupInformation taskUgi, SchedulerFragmentCompletingListener completionListener) {
+                            UserGroupInformation fsTaskUgi, SchedulerFragmentCompletingListener completionListener,
+                            SocketFactory socketFactory) {
     this.request = request;
     this.fragmentInfo = fragmentInfo;
     this.conf = conf;
@@ -153,8 +155,9 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
     this.fragmentCompletionHanler = fragmentCompleteHandler;
     this.tezHadoopShim = tezHadoopShim;
     this.initialEvent = initialEvent;
-    this.taskUgi = taskUgi;
+    this.fsTaskUgi = fsTaskUgi;
     this.completionListener = completionListener;
+    this.socketFactory = socketFactory;
   }
 
   public long getStartTime() {
@@ -196,27 +199,27 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
 
       // TODO Consolidate this code with TezChild.
       runtimeWatch.start();
-      if (taskUgi == null) {
-        taskUgi = UserGroupInformation.createRemoteUser(vertex.getUser());
+      if (fsTaskUgi == null) {
+        fsTaskUgi = UserGroupInformation.createRemoteUser(vertex.getUser());
       }
-      taskUgi.addCredentials(credentials);
+      fsTaskUgi.addCredentials(credentials);
 
       Map<String, ByteBuffer> serviceConsumerMetadata = new HashMap<>();
       serviceConsumerMetadata.put(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID,
           TezCommonUtils.convertJobTokenToBytes(jobToken));
       Multimap<String, String> startedInputsMap = createStartedInputMap(vertex);
 
-      UserGroupInformation taskOwner =
-          UserGroupInformation.createRemoteUser(vertex.getTokenIdentifier());
+      final UserGroupInformation taskOwner = fragmentInfo.getQueryInfo().getUmbilicalUgi();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("taskOwner hashCode:" + taskOwner.hashCode());
+      }
       final InetSocketAddress address =
           NetUtils.createSocketAddrForHost(request.getAmHost(), request.getAmPort());
-      SecurityUtil.setTokenService(jobToken, address);
-      taskOwner.addToken(jobToken);
       umbilical = taskOwner.doAs(new PrivilegedExceptionAction<LlapTaskUmbilicalProtocol>() {
         @Override
         public LlapTaskUmbilicalProtocol run() throws Exception {
           return RPC.getProxy(LlapTaskUmbilicalProtocol.class,
-              LlapTaskUmbilicalProtocol.versionID, address, conf);
+              LlapTaskUmbilicalProtocol.versionID, address, taskOwner, conf, socketFactory);
         }
       });
 
@@ -238,7 +241,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
       try {
         synchronized (this) {
           if (shouldRunTask) {
-            taskRunner = new TezTaskRunner2(conf, taskUgi, fragmentInfo.getLocalDirs(),
+            taskRunner = new TezTaskRunner2(conf, fsTaskUgi, fragmentInfo.getLocalDirs(),
                 taskSpec,
                 vertex.getQueryIdentifier().getAppAttemptNumber(),
                 serviceConsumerMetadata, envMap, startedInputsMap, taskReporter, executor,
@@ -260,7 +263,7 @@ public class TaskRunnerCallable extends CallableWithNdc<TaskRunner2Result> {
           isCompleted.set(true);
           return result;
         } finally {
-          FileSystem.closeAllForUGI(taskUgi);
+          FileSystem.closeAllForUGI(fsTaskUgi);
           LOG.info("ExecutionTime for Container: " + request.getContainerIdString() + "=" +
                   runtimeWatch.stop().elapsedMillis());
           if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b8d7192f/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
index 5dc1be5..ae3328a 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/impl/TaskExecutorTestHelpers.java
@@ -44,6 +44,8 @@ import org.apache.tez.runtime.task.TaskRunner2Result;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.net.SocketFactory;
+
 public class TaskExecutorTestHelpers {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestTaskExecutorService.class);
@@ -184,7 +186,7 @@ public class TaskExecutorTestHelpers {
           mock(KilledTaskHandler.class), mock(
               FragmentCompletionHandler.class), new DefaultHadoopShim(), null,
               requestProto.getWorkSpec().getVertex(), initialEvent, null, mock(
-              SchedulerFragmentCompletingListener.class));
+              SchedulerFragmentCompletingListener.class), mock(SocketFactory.class));
       this.workTime = workTime;
       this.canFinish = canFinish;
     }


[28/50] [abbrv] hive git commit: HIVE-15953 : better error messages for LLAP registry properties (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-15953 : better error messages for LLAP registry properties (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/af606ffd
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/af606ffd
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/af606ffd

Branch: refs/heads/hive-14535
Commit: af606ffd4bbd04a6def0132c8cfcba3d9067e44c
Parents: 32ab6c0
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Feb 21 13:45:04 2017 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Feb 21 13:45:04 2017 -0800

----------------------------------------------------------------------
 .../llap/registry/impl/LlapZookeeperRegistryImpl.java    | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/af606ffd/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
index 7ae80b0..0f8ff66 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapZookeeperRegistryImpl.java
@@ -451,9 +451,14 @@ public class LlapZookeeperRegistryImpl implements ServiceRegistry {
               AddressTypes.ADDRESS_PORT_FIELD));
       this.serviceAddress =
           RegistryTypeUtils.getAddressField(services.addresses.get(0), AddressTypes.ADDRESS_URI);
-      int memory = Integer.parseInt(srv.get(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname));
-      int vCores = Integer.parseInt(srv.get(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname));
-      this.resource = Resource.newInstance(memory, vCores);
+      String memStr = srv.get(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname, "");
+      String coreStr = srv.get(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname, "");
+      try {
+        this.resource = Resource.newInstance(Integer.parseInt(memStr), Integer.parseInt(coreStr));
+      } catch (NumberFormatException ex) {
+        throw new IOException("Invalid resource configuration for a LLAP node: memory "
+            + memStr + ", vcores " + coreStr);
+      }
     }
 
     @Override


[43/50] [abbrv] hive git commit: HIVE-16012 : BytesBytes hash table - better capacity exhaustion handling (Sergey Shelukhin, reviewed by Wei Zheng)

Posted by se...@apache.org.
HIVE-16012 : BytesBytes hash table - better capacity exhaustion handling (Sergey Shelukhin, reviewed by Wei Zheng)


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

Branch: refs/heads/hive-14535
Commit: 6ca79e3aa2b5a8812e5c4aaee80c4115e2b9def8
Parents: e941e63
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Feb 23 11:25:18 2017 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Feb 23 11:25:18 2017 -0800

----------------------------------------------------------------------
 .../hive/ql/exec/persistence/BytesBytesMultiHashMap.java  | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6ca79e3a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java
index 6b89e98..04e24bd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/BytesBytesMultiHashMap.java
@@ -26,6 +26,7 @@ import java.util.TreeMap;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.debug.Utils;
 import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput;
 import org.apache.hadoop.hive.serde2.SerDeException;
@@ -553,6 +554,12 @@ public final class BytesBytesMultiHashMap {
     if (capacity <= 0) {
       throw new AssertionError("Invalid capacity " + capacity);
     }
+    if (capacity > Integer.MAX_VALUE) {
+      throw new RuntimeException("Attempting to expand the hash table to " + capacity
+          + " that overflows maximum array size. For this query, you may want to disable "
+          + ConfVars.HIVEDYNAMICPARTITIONHASHJOIN.varname + " or reduce "
+          + ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD.varname);
+    }
   }
 
   /**
@@ -715,8 +722,7 @@ public final class BytesBytesMultiHashMap {
   }
 
   private void expandAndRehash() {
-    long capacity = refs.length << 1;
-    expandAndRehashImpl(capacity);
+    expandAndRehashImpl(((long)refs.length) << 1);
   }
 
   private void expandAndRehashImpl(long capacity) {


[08/50] [abbrv] hive git commit: HIVE-15950 Make DbTxnManager use Metastore client consistently with callers (Eugene Koifman, reviewed by Vaibhav Gumashta)

Posted by se...@apache.org.
HIVE-15950 Make DbTxnManager use Metastore client consistently with callers (Eugene Koifman, 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/bb4d8db5
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bb4d8db5
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bb4d8db5

Branch: refs/heads/hive-14535
Commit: bb4d8db5093984b94d3bb996e286e10a0dc2bef3
Parents: 3485d02
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Fri Feb 17 09:32:15 2017 -0800
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Fri Feb 17 09:32:15 2017 -0800

----------------------------------------------------------------------
 .../hive/metastore/RetryingMetaStoreClient.java |   4 +-
 .../hadoop/hive/ql/lockmgr/DbLockManager.java   |  28 ++--
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    | 144 +++++++------------
 3 files changed, 68 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bb4d8db5/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
index a6545a9..d3e5f7e 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
@@ -215,8 +215,8 @@ public class RetryingMetaStoreClient implements InvocationHandler {
         throw caughtException;
       }
       retriesMade++;
-      LOG.warn("MetaStoreClient lost connection. Attempting to reconnect.",
-          caughtException);
+      LOG.warn("MetaStoreClient lost connection. Attempting to reconnect (" + retriesMade + " of " +
+          retryLimit + ") after " + retryDelaySeconds + "s. " + method.getName(), caughtException);
       Thread.sleep(retryDelaySeconds * 1000);
     }
     return ret;

http://git-wip-us.apache.org/repos/asf/hive/blob/bb4d8db5/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
index 529e64c..c3725ad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.SynchronizedMetaStoreClient;
 import org.apache.hadoop.hive.ql.exec.DDLTask;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,10 +42,10 @@ import java.util.concurrent.TimeUnit;
 
 /**
  * An implementation of HiveLockManager for use with {@link org.apache.hadoop.hive.ql.lockmgr.DbTxnManager}.
- * Note, this lock manager is not meant to stand alone.  It cannot be used
- * without the DbTxnManager.
+ * Note, this lock manager is not meant to be stand alone.  It cannot be used without the DbTxnManager.
+ * See {@link DbTxnManager#getMS()} for important concurrency/metastore access notes.
  */
-public class DbLockManager implements HiveLockManager{
+public final class DbLockManager implements HiveLockManager{
 
   static final private String CLASS_NAME = DbLockManager.class.getName();
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
@@ -54,14 +53,14 @@ public class DbLockManager implements HiveLockManager{
   private long MAX_SLEEP;
   //longer term we should always have a txn id and then we won't need to track locks here at all
   private Set<DbHiveLock> locks;
-  private SynchronizedMetaStoreClient client;
   private long nextSleep = 50;
   private final HiveConf conf;
+  private final DbTxnManager txnManager;
 
-  DbLockManager(SynchronizedMetaStoreClient client, HiveConf conf) {
+  DbLockManager(HiveConf conf, DbTxnManager txnManager) {
     locks = new HashSet<>();
-    this.client = client;
     this.conf = conf;
+    this.txnManager = txnManager;
   }
 
   @Override
@@ -100,7 +99,7 @@ public class DbLockManager implements HiveLockManager{
     int maxNumWaits = Math.max(0, conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_NUMRETRIES));
     try {
       LOG.info("Requesting: queryId=" + queryId + " " + lock);
-      LockResponse res = client.lock(lock);
+      LockResponse res = txnManager.getMS().lock(lock);
       //link lockId to queryId
       LOG.info("Response to queryId=" + queryId + " " + res);
       if(!isBlocking) {
@@ -112,8 +111,7 @@ public class DbLockManager implements HiveLockManager{
       long startRetry = System.currentTimeMillis();
       while (res.getState() == LockState.WAITING && numRetries++ < maxNumWaits) {
         backoff();
-        res = client.checkLock(res.getLockid());
-
+        res = txnManager.getMS().checkLock(res.getLockid());
       }
       long retryDuration = System.currentTimeMillis() - startRetry;
       DbHiveLock hl = new DbHiveLock(res.getLockid(), queryId, lock.getTxnid());
@@ -203,7 +201,7 @@ public class DbLockManager implements HiveLockManager{
    */
   LockState checkLock(long extLockId) throws LockException {
     try {
-      return client.checkLock(extLockId).getState();
+      return txnManager.getMS().checkLock(extLockId).getState();
     } catch (TException e) {
       throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(),
         e);
@@ -216,7 +214,7 @@ public class DbLockManager implements HiveLockManager{
     boolean removed = false;
     try {
       LOG.debug("Unlocking " + hiveLock);
-      client.unlock(lockId);
+      txnManager.getMS().unlock(lockId);
       //important to remove after unlock() in case it fails
       removed = locks.remove(hiveLock);
       Metrics metrics = MetricsFactory.getInstance();
@@ -283,7 +281,7 @@ public class DbLockManager implements HiveLockManager{
 
   public ShowLocksResponse getLocks(ShowLocksRequest showLocksRequest) throws LockException {
     try {
-      return client.showLocks(showLocksRequest);
+      return txnManager.getMS().showLocks(showLocksRequest);
     } catch (TException e) {
       throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), e);
     }
@@ -354,8 +352,8 @@ public class DbLockManager implements HiveLockManager{
   /**
    * Clear the memory of the locks in this object.  This won't clear the locks from the database.
    * It is for use with
-   * {@link #DbLockManager(org.apache.hadoop.hive.metastore.IMetaStoreClient, org.apache.hadoop.hive.conf.HiveConf)} .commitTxn} and
-   * {@link #DbLockManager(org.apache.hadoop.hive.metastore.IMetaStoreClient, org.apache.hadoop.hive.conf.HiveConf)} .rollbackTxn}.
+   * {@link #DbLockManager(HiveConf, DbTxnManager)} .commitTxn} and
+   * {@link #DbLockManager(HiveConf, DbTxnManager)} .rollbackTxn}.
    */
   void clearLocalLockRecords() {
     locks.clear();

http://git-wip-us.apache.org/repos/asf/hive/blob/bb4d8db5/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index a985eb1..62f7c5a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hive.ql.lockmgr;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.SynchronizedMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hive.common.util.ShutdownHookManager;
 import org.slf4j.Logger;
@@ -52,21 +52,23 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 /**
- * An implementation of HiveTxnManager that stores the transactions in the
- * metastore database.
+ * An implementation of HiveTxnManager that stores the transactions in the metastore database.
+ * There should be 1 instance o {@link DbTxnManager} per {@link org.apache.hadoop.hive.ql.session.SessionState}
+ * with a single thread accessing it at a time, with the exception of {@link #heartbeat()} method.
+ * The later may (usually will) be called from a timer thread.
+ * See {@link #getMS()} for more important concurrency/metastore access notes.
  */
-public class DbTxnManager extends HiveTxnManagerImpl {
+public final class DbTxnManager extends HiveTxnManagerImpl {
 
   static final private String CLASS_NAME = DbTxnManager.class.getName();
   static final private Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
 
-  private DbLockManager lockMgr = null;
-  private SynchronizedMetaStoreClient client = null;
+  private volatile DbLockManager lockMgr = null;
   /**
    * The Metastore NEXT_TXN_ID.NTXN_NEXT is initialized to 1; it contains the next available
    * transaction id.  Thus is 1 is first transaction id.
    */
-  private long txnId = 0;
+  private volatile long txnId = 0;
   /**
    * assigns a unique monotonically increasing ID to each statement
    * which is part of an open transaction.  This is used by storage
@@ -84,33 +86,31 @@ public class DbTxnManager extends HiveTxnManagerImpl {
   private ScheduledFuture<?> heartbeatTask = null;
   private Runnable shutdownRunner = null;
   private static final int SHUTDOWN_HOOK_PRIORITY = 0;
-
-  // SynchronizedMetaStoreClient object per heartbeater thread.
-  private static ThreadLocal<SynchronizedMetaStoreClient> threadLocalMSClient =
-      new ThreadLocal<SynchronizedMetaStoreClient>() {
-
-        @Override
-        protected SynchronizedMetaStoreClient initialValue() {
-          return null;
-        }
-
-        @Override
-        public synchronized void remove() {
-          SynchronizedMetaStoreClient client = this.get();
-          if (client != null) {
-            client.close();
-          }
-          super.remove();
-        }
-      };
-
-  private static AtomicInteger heartbeaterMSClientCount = new AtomicInteger(0);
-  private static int heartbeaterThreadPoolSize = 0;
-
-  private static SynchronizedMetaStoreClient getThreadLocalMSClient() {
-    return threadLocalMSClient.get();
+  /**
+   * We do this on every call to make sure TM uses same MS connection as is used by the caller (Driver,
+   * SemanticAnalyzer, etc).  {@code Hive} instances are cached using ThreadLocal and
+   * {@link IMetaStoreClient} is cached within {@code Hive} with additional logic.  Futhermore, this
+   * ensures that multiple threads are not sharing the same Thrift client (which could happen
+   * if we had cached {@link IMetaStoreClient} here.
+   *
+   * ThreadLocal gets cleaned up automatically when its thread goes away
+   * https://docs.oracle.com/javase/7/docs/api/java/lang/ThreadLocal.html.  This is especially
+   * important for threads created by {@link #heartbeatExecutorService} threads.
+   *
+   * Embedded {@link DbLockManager} follows the same logic.
+   * @return IMetaStoreClient
+   * @throws LockException on any errors
+   */
+  IMetaStoreClient getMS() throws LockException {
+    try {
+      return Hive.get(conf).getMSC();
+    }
+    catch(HiveException|MetaException e) {
+      String msg = "Unable to reach Hive Metastore: " + e.getMessage();
+      LOG.error(msg, e);
+      throw new LockException(e);
+    }
   }
-
   DbTxnManager() {
     shutdownRunner = new Runnable() {
       @Override
@@ -148,7 +148,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
       throw new LockException("Transaction already opened. " + JavaUtils.txnIdToString(txnId));
     }
     try {
-      txnId = client.openTxn(user);
+      txnId = getMS().openTxn(user);
       statementId = 0;
       LOG.debug("Opened " + JavaUtils.txnIdToString(txnId));
       ctx.setHeartbeater(startHeartbeat(delay));
@@ -158,11 +158,15 @@ public class DbTxnManager extends HiveTxnManagerImpl {
     }
   }
 
+  /**
+   * we don't expect multiple thread to call this method concurrently but {@link #lockMgr} will
+   * be read by a different threads that one writing it, thus it's {@code volatile}
+   */
   @Override
   public HiveLockManager getLockManager() throws LockException {
     init();
     if (lockMgr == null) {
-      lockMgr = new DbLockManager(client, conf);
+      lockMgr = new DbLockManager(conf, this);
     }
     return lockMgr;
   }
@@ -388,7 +392,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
       lockMgr.clearLocalLockRecords();
       stopHeartbeat();
       LOG.debug("Committing txn " + JavaUtils.txnIdToString(txnId));
-      client.commitTxn(txnId);
+      getMS().commitTxn(txnId);
     } catch (NoSuchTxnException e) {
       LOG.error("Metastore could not find " + JavaUtils.txnIdToString(txnId));
       throw new LockException(e, ErrorMsg.TXN_NO_SUCH_TRANSACTION, JavaUtils.txnIdToString(txnId));
@@ -414,7 +418,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
       lockMgr.clearLocalLockRecords();
       stopHeartbeat();
       LOG.debug("Rolling back " + JavaUtils.txnIdToString(txnId));
-      client.rollbackTxn(txnId);
+      getMS().rollbackTxn(txnId);
     } catch (NoSuchTxnException e) {
       LOG.error("Metastore could not find " + JavaUtils.txnIdToString(txnId));
       throw new LockException(e, ErrorMsg.TXN_NO_SUCH_TRANSACTION, JavaUtils.txnIdToString(txnId));
@@ -460,29 +464,11 @@ public class DbTxnManager extends HiveTxnManagerImpl {
     for (HiveLock lock : locks) {
       long lockId = ((DbLockManager.DbHiveLock)lock).lockId;
       try {
-        // Get the threadlocal metastore client for the heartbeat calls.
-        SynchronizedMetaStoreClient heartbeaterClient = getThreadLocalMSClient();
-        if (heartbeaterClient == null) {
-          Hive db;
-          try {
-            db = Hive.get(conf);
-            // Create a new threadlocal synchronized metastore client for use in heartbeater threads.
-            // This makes the concurrent use of heartbeat thread safe, and won't cause transaction
-            // abort due to a long metastore client call blocking the heartbeat call.
-            heartbeaterClient = new SynchronizedMetaStoreClient(db.getMSC());
-            threadLocalMSClient.set(heartbeaterClient);
-          } catch (HiveException e) {
-            LOG.error("Unable to create new metastore client for heartbeating", e);
-            throw new LockException(e);
-          }
-          // Increment the threadlocal metastore client count
-          if (heartbeaterMSClientCount.incrementAndGet() >= heartbeaterThreadPoolSize) {
-            LOG.warn("The number of heartbeater metastore clients - + "
-                + heartbeaterMSClientCount.get() + ", has exceeded the max limit - "
-                + heartbeaterThreadPoolSize);
-          }
-        }
-        heartbeaterClient.heartbeat(txnId, lockId);
+        /**
+         * This relies on the ThreadLocal caching, which implies that the same {@link IMetaStoreClient},
+         * in particular the Thrift connection it uses is never shared between threads
+         */
+        getMS().heartbeat(txnId, lockId);
       } catch (NoSuchLockException e) {
         LOG.error("Unable to find lock " + JavaUtils.lockIdToString(lockId));
         throw new LockException(e, ErrorMsg.LOCK_NO_SUCH_LOCK, JavaUtils.lockIdToString(lockId));
@@ -554,7 +540,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
   public ValidTxnList getValidTxns() throws LockException {
     init();
     try {
-      return client.getValidTxns(txnId);
+      return getMS().getValidTxns(txnId);
     } catch (TException e) {
       throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(),
           e);
@@ -598,21 +584,10 @@ public class DbTxnManager extends HiveTxnManagerImpl {
   }
 
   private void init() throws LockException {
-    if (client == null) {
-      if (conf == null) {
-        throw new RuntimeException("Must call setHiveConf before any other " +
-            "methods.");
-      }
-      try {
-        Hive db = Hive.get(conf);
-        client = new SynchronizedMetaStoreClient(db.getMSC());
-        initHeartbeatExecutorService();
-      } catch (MetaException e) {
-        throw new LockException(ErrorMsg.METASTORE_COULD_NOT_INITIATE.getMsg(), e);
-      } catch (HiveException e) {
-        throw new LockException(ErrorMsg.METASTORE_COULD_NOT_INITIATE.getMsg(), e);
-      }
+    if (conf == null) {
+      throw new RuntimeException("Must call setHiveConf before any other methods.");
     }
+    initHeartbeatExecutorService();
   }
 
   private synchronized void initHeartbeatExecutorService() {
@@ -620,10 +595,9 @@ public class DbTxnManager extends HiveTxnManagerImpl {
         && !heartbeatExecutorService.isTerminated()) {
       return;
     }
-    heartbeaterThreadPoolSize =
-        conf.getIntVar(HiveConf.ConfVars.HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE);
     heartbeatExecutorService =
-        Executors.newScheduledThreadPool(heartbeaterThreadPoolSize, new ThreadFactory() {
+        Executors.newScheduledThreadPool(
+          conf.getIntVar(HiveConf.ConfVars.HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE), new ThreadFactory() {
           private final AtomicInteger threadCounter = new AtomicInteger();
 
           @Override
@@ -635,22 +609,10 @@ public class DbTxnManager extends HiveTxnManagerImpl {
   }
 
   public static class HeartbeaterThread extends Thread {
-    public HeartbeaterThread(Runnable target, String name) {
+    HeartbeaterThread(Runnable target, String name) {
       super(target, name);
       setDaemon(true);
     }
-
-    @Override
-    /**
-     * We're overriding finalize so that we can do an orderly cleanup of resources held by
-     * the threadlocal metastore client.
-     */
-    protected void finalize() throws Throwable {
-      threadLocalMSClient.remove();
-      // Adjust the metastore client count
-      DbTxnManager.heartbeaterMSClientCount.decrementAndGet();
-      super.finalize();
-    }
   }
 
   @Override


[42/50] [abbrv] hive git commit: HIVE-12492: MapJoin: 4 million unique integers seems to be a probe plateau (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-12492: MapJoin: 4 million unique integers seems to be a probe plateau (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/e941e63c
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e941e63c
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e941e63c

Branch: refs/heads/hive-14535
Commit: e941e63c7d2830395e0b535e9b1a3c33d6e5b652
Parents: 759766e
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Thu Feb 23 08:50:20 2017 +0000
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu Feb 23 18:50:14 2017 +0000

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   5 +
 .../test/resources/testconfiguration.properties |   1 +
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   | 106 +++-
 .../stats/annotation/StatsRulesProcFactory.java |  44 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |   6 +-
 .../queries/clientpositive/join_max_hashtable.q |  37 ++
 .../llap/join_max_hashtable.q.out               | 490 +++++++++++++++++++
 7 files changed, 678 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e941e63c/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 0b315e1..46be3fb 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1422,6 +1422,11 @@ public class HiveConf extends Configuration {
         "This controls how many partitions can be scanned for each partitioned table.\n" +
         "The default value \"-1\" means no limit. (DEPRECATED: Please use " + ConfVars.METASTORE_LIMIT_PARTITION_REQUEST + " in the metastore instead.)"),
 
+    HIVECONVERTJOINMAXENTRIESHASHTABLE("hive.auto.convert.join.hashtable.max.entries", 4194304L,
+        "If hive.auto.convert.join.noconditionaltask is off, this parameter does not take affect. \n" +
+        "However, if it is on, and the predicated number of entries in hashtable for a given join \n" +
+        "input is larger than this number, the join will not be converted to a mapjoin. \n" +
+        "The value \"-1\" means no limit."),
     HIVEHASHTABLEKEYCOUNTADJUSTMENT("hive.hashtable.key.count.adjustment", 1.0f,
         "Adjustment to mapjoin hashtable size derived from table and column statistics; the estimate" +
         " of the number of keys is divided by this value. If the value is 0, statistics are not used" +

http://git-wip-us.apache.org/repos/asf/hive/blob/e941e63c/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index d344464..5b30157 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -501,6 +501,7 @@ minillaplocal.query.files=acid_globallimit.q,\
   join1.q,\
   join_acid_non_acid.q,\
   join_filters.q,\
+  join_max_hashtable.q,\
   join_nulls.q,\
   join_nullsafe.q,\
   leftsemijoin_mr.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/e941e63c/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index 93e3631..e68618a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.optimizer;
 
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -53,6 +52,7 @@ import org.apache.hadoop.hive.ql.parse.GenTezUtils;
 import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ColStatistics;
 import org.apache.hadoop.hive.ql.plan.CommonMergeJoinDesc;
 import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -63,6 +63,7 @@ import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
 import org.apache.hadoop.hive.ql.plan.OpTraits;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.Statistics;
+import org.apache.hadoop.hive.ql.stats.StatsUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -117,7 +118,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
       numBuckets = 1;
     }
     LOG.info("Estimated number of buckets " + numBuckets);
-    int mapJoinConversionPos = getMapJoinConversionPos(joinOp, context, numBuckets, false, maxSize);
+    int mapJoinConversionPos = getMapJoinConversionPos(joinOp, context, numBuckets, false, maxSize, true);
     if (mapJoinConversionPos < 0) {
       Object retval = checkAndConvertSMBJoin(context, joinOp, tezBucketJoinProcCtx);
       if (retval == null) {
@@ -141,7 +142,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     // check if we can convert to map join no bucket scaling.
     LOG.info("Convert to non-bucketed map join");
     if (numBuckets != 1) {
-      mapJoinConversionPos = getMapJoinConversionPos(joinOp, context, 1, false, maxSize);
+      mapJoinConversionPos = getMapJoinConversionPos(joinOp, context, 1, false, maxSize, true);
     }
     if (mapJoinConversionPos < 0) {
       // we are just converting to a common merge join operator. The shuffle
@@ -519,8 +520,22 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     return false;
   }
 
+  /**
+   * Obtain big table position for join.
+   *
+   * @param joinOp join operator
+   * @param context optimization context
+   * @param buckets bucket count for Bucket Map Join conversion consideration or reduce count
+   * for Dynamic Hash Join conversion consideration
+   * @param skipJoinTypeChecks whether to skip join type checking
+   * @param maxSize size threshold for Map Join conversion
+   * @param checkHashTableEntries whether to check threshold for distinct keys in hash table for Map Join
+   * @return returns big table position or -1 if it cannot be determined
+   * @throws SemanticException
+   */
   public int getMapJoinConversionPos(JoinOperator joinOp, OptimizeTezProcContext context,
-      int buckets, boolean skipJoinTypeChecks, long maxSize) throws SemanticException {
+      int buckets, boolean skipJoinTypeChecks, long maxSize, boolean checkHashTableEntries)
+              throws SemanticException {
     if (!skipJoinTypeChecks) {
       /*
        * HIVE-9038: Join tests fail in tez when we have more than 1 join on the same key and there is
@@ -628,10 +643,20 @@ public class ConvertJoinMapJoin implements NodeProcessor {
         // We are replacing the current big table with a new one, thus
         // we need to count the current one as a map table then.
         totalSize += bigInputStat.getDataSize();
+        // Check if number of distinct keys is larger than given max
+        // number of entries for HashMap. If it is, we do not convert.
+        if (checkHashTableEntries && !checkNumberOfEntriesForHashTable(joinOp, bigTablePosition, context)) {
+          return -1;
+        }
       } else if (!selectedBigTable) {
         // This is not the first table and we are not using it as big table,
         // in fact, we're adding this table as a map table
         totalSize += inputSize;
+        // Check if number of distinct keys is larger than given max
+        // number of entries for HashMap. If it is, we do not convert.
+        if (checkHashTableEntries && !checkNumberOfEntriesForHashTable(joinOp, pos, context)) {
+          return -1;
+        }
       }
 
       if (totalSize/buckets > maxSize) {
@@ -905,8 +930,8 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     int numReducers = estimateNumBuckets(joinOp, false);
     LOG.info("Try dynamic partitioned hash join with estimated " + numReducers + " reducers");
     int bigTablePos = getMapJoinConversionPos(joinOp, context, numReducers, false,
-                          context.conf.getLongVar(
-                              HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD));
+            context.conf.getLongVar(HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD),
+            false);
     if (bigTablePos >= 0) {
       // Now that we have the big table index, get real numReducers value based on big table RS
       ReduceSinkOperator bigTableParentRS =
@@ -951,7 +976,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     }
 
     int pos = getMapJoinConversionPos(joinOp, context, estimateNumBuckets(joinOp, false),
-                  true, Long.MAX_VALUE);
+                  true, Long.MAX_VALUE, false);
     if (pos < 0) {
       LOG.info("Could not get a valid join position. Defaulting to position 0");
       pos = 0;
@@ -961,4 +986,71 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     LOG.info("Fallback to common merge join operator");
     convertJoinSMBJoin(joinOp, context, pos, 0, false);
   }
+
+  /* Returns true if it passes the test, false otherwise. */
+  private boolean checkNumberOfEntriesForHashTable(JoinOperator joinOp, int position,
+          OptimizeTezProcContext context) {
+    long max = HiveConf.getLongVar(context.parseContext.getConf(),
+            HiveConf.ConfVars.HIVECONVERTJOINMAXENTRIESHASHTABLE);
+    if (max < 1) {
+      // Max is disabled, we can safely return true
+      return true;
+    }
+    // Calculate number of different entries and evaluate
+    ReduceSinkOperator rsOp = (ReduceSinkOperator) joinOp.getParentOperators().get(position);
+    List<String> keys = StatsUtils.getQualifedReducerKeyNames(rsOp.getConf().getOutputKeyColumnNames());
+    Statistics inputStats = rsOp.getStatistics();
+    List<ColStatistics> columnStats = new ArrayList<>();
+    for (String key : keys) {
+      ColStatistics cs = inputStats.getColumnStatisticsFromColName(key);
+      if (cs == null) {
+        LOG.debug("Couldn't get statistics for: {}", key);
+        return true;
+      }
+      columnStats.add(cs);
+    }
+    long numRows = inputStats.getNumRows();
+    long estimation = estimateNDV(numRows, columnStats);
+    LOG.debug("Estimated NDV for input {}: {}; Max NDV for MapJoin conversion: {}",
+            position, estimation, max);
+    if (estimation > max) {
+      // Estimation larger than max
+      LOG.debug("Number of different entries for HashTable is greater than the max; "
+          + "we do not converting to MapJoin");
+      return false;
+    }
+    // We can proceed with the conversion
+    return true;
+  }
+
+  private static long estimateNDV(long numRows, List<ColStatistics> columnStats) {
+    // If there is a single column, return the number of distinct values
+    if (columnStats.size() == 1) {
+      return columnStats.get(0).getCountDistint();
+    }
+
+    // The expected number of distinct values when choosing p values
+    // with replacement from n integers is n . (1 - ((n - 1) / n) ^ p).
+    //
+    // If we have several uniformly distributed attributes A1 ... Am
+    // with N1 ... Nm distinct values, they behave as one uniformly
+    // distributed attribute with N1 * ... * Nm distinct values.
+    long n = 1L;
+    for (ColStatistics cs : columnStats) {
+      final long ndv = cs.getCountDistint();
+      if (ndv > 1) {
+        n = StatsUtils.safeMult(n, ndv);
+      }
+    }
+    final double nn = (double) n;
+    final double a = (nn - 1d) / nn;
+    if (a == 1d) {
+      // A under-flows if nn is large.
+      return numRows;
+    }
+    final double v = nn * (1d - Math.pow(a, numRows));
+    // Cap at fact-row-count, because numerical artifacts can cause it
+    // to go a few % over.
+    return Math.min(Math.round(v), numRows);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e941e63c/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
index 61f1374..bdb09a8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
@@ -51,10 +51,43 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.plan.*;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.ColStatistics;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnListDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc.ExprNodeDescEqualityWrapper;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDynamicListDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDynamicValueDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.GroupByDesc;
+import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
+import org.apache.hadoop.hive.ql.plan.JoinDesc;
+import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.Statistics;
 import org.apache.hadoop.hive.ql.stats.StatsUtils;
-import org.apache.hadoop.hive.ql.udf.generic.*;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFInBloomFilter;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualNS;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNot;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFStruct;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
@@ -1409,7 +1442,12 @@ public class StatsRulesProcFactory {
         // get the join keys from parent ReduceSink operators
         for (int pos = 0; pos < parents.size(); pos++) {
           ReduceSinkOperator parent = (ReduceSinkOperator) jop.getParentOperators().get(pos);
-          Statistics parentStats = parent.getStatistics();
+          Statistics parentStats;
+          try {
+            parentStats = parent.getStatistics().clone();
+          } catch (CloneNotSupportedException e) {
+            throw new SemanticException(ErrorMsg.STATISTICS_CLONING_FAILED.getMsg());
+          }
           keyExprs = StatsUtils.getQualifedReducerKeyNames(parent.getConf()
               .getOutputKeyColumnNames());
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e941e63c/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index 0da7ea4..e48b609 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -1278,7 +1278,11 @@ public class StatsUtils {
         ColStatistics colStats = parentStats.getColumnStatisticsFromColName(colName);
         if (colStats != null) {
           /* If statistics for the column already exist use it. */
-          return colStats;
+          try {
+            return colStats.clone();
+          } catch (CloneNotSupportedException e) {
+            return null;
+          }
         }
 
         // virtual columns

http://git-wip-us.apache.org/repos/asf/hive/blob/e941e63c/ql/src/test/queries/clientpositive/join_max_hashtable.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/join_max_hashtable.q b/ql/src/test/queries/clientpositive/join_max_hashtable.q
new file mode 100644
index 0000000..9c30a0d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/join_max_hashtable.q
@@ -0,0 +1,37 @@
+set hive.auto.convert.join=true;
+set hive.optimize.dynamic.partition.hashjoin=true;
+set hive.auto.convert.join.hashtable.max.entries=500;
+
+-- CONVERT
+EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key);
+
+-- CONVERT
+EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key AND x.value = y.value);
+
+set hive.auto.convert.join.hashtable.max.entries=300;
+
+-- CONVERT
+EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key);
+
+-- DO NOT CONVERT
+EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key AND x.value = y.value);
+
+set hive.auto.convert.join.hashtable.max.entries=10;
+
+-- DO NOT CONVERT
+EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key);
+
+-- DO NOT CONVERT
+EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key AND x.value = y.value);

http://git-wip-us.apache.org/repos/asf/hive/blob/e941e63c/ql/src/test/results/clientpositive/llap/join_max_hashtable.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/join_max_hashtable.q.out b/ql/src/test/results/clientpositive/llap/join_max_hashtable.q.out
new file mode 100644
index 0000000..85d45fe
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/join_max_hashtable.q.out
@@ -0,0 +1,490 @@
+PREHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: x
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0, _col1
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 1219 Data size: 216982 Basic stats: COMPLETE Column stats: COMPLETE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 1219 Data size: 216982 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
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: y
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: no inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key AND x.value = y.value)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key AND x.value = y.value)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: x
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key is not null and value is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string), _col1 (type: string)
+                          1 _col0 (type: string), _col1 (type: string)
+                        outputColumnNames: _col0, _col1
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 5 Data size: 890 Basic stats: COMPLETE Column stats: COMPLETE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 5 Data size: 890 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
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: y
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key is not null and value is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: no inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: x
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0, _col1
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 1219 Data size: 216982 Basic stats: COMPLETE Column stats: COMPLETE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 1219 Data size: 216982 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
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: y
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: no inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key AND x.value = y.value)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key AND x.value = y.value)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: x
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key is not null and value is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: y
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key is not null and value is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
+                  1 KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
+                outputColumnNames: _col0, _col1
+                input vertices:
+                  1 Map 3
+                Statistics: Num rows: 5 Data size: 890 Basic stats: COMPLETE Column stats: COMPLETE
+                HybridGraceHashJoin: true
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 5 Data size: 890 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: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: x
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: y
+                  Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 KEY.reducesinkkey0 (type: string)
+                  1 KEY.reducesinkkey0 (type: string)
+                outputColumnNames: _col0, _col1
+                input vertices:
+                  1 Map 3
+                Statistics: Num rows: 1219 Data size: 216982 Basic stats: COMPLETE Column stats: COMPLETE
+                HybridGraceHashJoin: true
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1219 Data size: 216982 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: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key AND x.value = y.value)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT x.key, x.value
+FROM src x JOIN src y ON (x.key = y.key AND x.value = y.value)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: x
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key is not null and value is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: y
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key is not null and value is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
+                  1 KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
+                outputColumnNames: _col0, _col1
+                input vertices:
+                  1 Map 3
+                Statistics: Num rows: 5 Data size: 890 Basic stats: COMPLETE Column stats: COMPLETE
+                HybridGraceHashJoin: true
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 5 Data size: 890 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
+


[27/50] [abbrv] hive git commit: HIVE-15948 : Failing test: TestCliDriver, TestSparkCliDriver join31 (Sahil Takiar via Rui Li)

Posted by se...@apache.org.
HIVE-15948 : Failing test: TestCliDriver, TestSparkCliDriver join31 (Sahil Takiar via Rui Li)

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


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

Branch: refs/heads/hive-14535
Commit: 32ab6c08239712ef1a6ca5ab2e70e3ba25105469
Parents: e17a040
Author: Sahil Takiar <ta...@gmail.com>
Authored: Tue Feb 21 13:10:24 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Tue Feb 21 13:10:24 2017 -0800

----------------------------------------------------------------------
 ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q | 2 +-
 ql/src/test/queries/clientpositive/join31.q            | 2 +-
 ql/src/test/queries/clientpositive/multiMapJoin2.q     | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/32ab6c08/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q b/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
index 8936073..e9e434c 100644
--- a/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
+++ b/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
@@ -1,6 +1,6 @@
 set hive.cbo.returnpath.hiveop=true;
 set hive.stats.fetch.column.stats=true;
-set hive.enable.semijoin.conversion=true;
+set hive.optimize.semijoin.conversion=true;
 ;
 
 set hive.exec.reducers.max = 1;

http://git-wip-us.apache.org/repos/asf/hive/blob/32ab6c08/ql/src/test/queries/clientpositive/join31.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/join31.q b/ql/src/test/queries/clientpositive/join31.q
index aa17b4d..62407a0 100644
--- a/ql/src/test/queries/clientpositive/join31.q
+++ b/ql/src/test/queries/clientpositive/join31.q
@@ -1,5 +1,5 @@
 set hive.mapred.mode=nonstrict;
-set hive.enable.semijoin.conversion=true;
+set hive.optimize.semijoin.conversion=true;
 -- SORT_QUERY_RESULTS
 
 CREATE TABLE dest_j1(key STRING, cnt INT);

http://git-wip-us.apache.org/repos/asf/hive/blob/32ab6c08/ql/src/test/queries/clientpositive/multiMapJoin2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/multiMapJoin2.q b/ql/src/test/queries/clientpositive/multiMapJoin2.q
index 38ab1a7..14b2eda 100644
--- a/ql/src/test/queries/clientpositive/multiMapJoin2.q
+++ b/ql/src/test/queries/clientpositive/multiMapJoin2.q
@@ -3,7 +3,7 @@ set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.
 set hive.auto.convert.join=true;
 set hive.auto.convert.join.noconditionaltask=true;
 set hive.auto.convert.join.noconditionaltask.size=6000;
-set hive.enable.semijoin.conversion=true;
+set hive.optimize.semijoin.conversion=true;
 
 -- we will generate one MR job.
 EXPLAIN


[26/50] [abbrv] hive git commit: HIVE-15847 : In Progress update refreshes seem slow (Anishek Agarwal, via THejas Nair)

Posted by se...@apache.org.
HIVE-15847 : In Progress update refreshes seem slow (Anishek Agarwal, 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/e17a0409
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e17a0409
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e17a0409

Branch: refs/heads/hive-14535
Commit: e17a0409c920bcf75d8915047c3318d621e2ced0
Parents: 7fa8e37
Author: Anishek Agarwal <an...@gmail.com>
Authored: Tue Feb 21 12:48:19 2017 -0800
Committer: Thejas M Nair <th...@hortonworks.com>
Committed: Tue Feb 21 12:48:25 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hive/common/log/InPlaceUpdate.java   |   1 +
 .../hive/ql/exec/tez/monitoring/DAGSummary.java |  12 +-
 .../ql/exec/tez/monitoring/RenderStrategy.java  | 154 +++++++++++++++++++
 .../ql/exec/tez/monitoring/TezJobMonitor.java   | 110 ++-----------
 4 files changed, 168 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e17a0409/common/src/java/org/apache/hadoop/hive/common/log/InPlaceUpdate.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/log/InPlaceUpdate.java b/common/src/java/org/apache/hadoop/hive/common/log/InPlaceUpdate.java
index bfdb4fa..6db5c18 100644
--- a/common/src/java/org/apache/hadoop/hive/common/log/InPlaceUpdate.java
+++ b/common/src/java/org/apache/hadoop/hive/common/log/InPlaceUpdate.java
@@ -163,6 +163,7 @@ public class InPlaceUpdate {
       progressStr,
       elapsedTime);
 
+    reprintLine(SEPARATOR);
     reprintLineWithColorAsBold(footer, Ansi.Color.RED);
     reprintLine(SEPARATOR);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/e17a0409/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java
index 5840ad6..1400be4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/DAGSummary.java
@@ -26,16 +26,11 @@ import java.util.*;
 
 class DAGSummary implements PrintSummary {
 
-  private static final int FILE_HEADER_SEPARATOR_WIDTH = InPlaceUpdate.MIN_TERMINAL_WIDTH + 34;
-  private static final String FILE_HEADER_SEPARATOR = new String(new char[FILE_HEADER_SEPARATOR_WIDTH]).replace("\0", "-");
-
-  private static final String FORMATTING_PATTERN = "%10s %12s %16s %13s %14s %13s %12s %14s %15s";
+  private static final String FILE_HEADER_SEPARATOR = new String(new char[InPlaceUpdate.MIN_TERMINAL_WIDTH]).replace("\0", "-");
+  private static final String FORMATTING_PATTERN = "%10s %17s %14s %14s %15s %16s";
   private static final String FILE_HEADER = String.format(
       FORMATTING_PATTERN,
       "VERTICES",
-      "TOTAL_TASKS",
-      "FAILED_ATTEMPTS",
-      "KILLED_TASKS",
       "DURATION(ms)",
       "CPU_TIME(ms)",
       "GC_TIME(ms)",
@@ -170,9 +165,6 @@ class DAGSummary implements PrintSummary {
 
     return String.format(FORMATTING_PATTERN,
         vertexName,
-        progress.getTotalTaskCount(),
-        progress.getFailedTaskAttemptCount(),
-        progress.getKilledTaskAttemptCount(),
         secondsFormatter.format((duration)),
         commaFormatter.format(cpuTimeMillis),
         commaFormatter.format(gcTimeMillis),

http://git-wip-us.apache.org/repos/asf/hive/blob/e17a0409/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/RenderStrategy.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/RenderStrategy.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/RenderStrategy.java
new file mode 100644
index 0000000..bb9a5e7
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/RenderStrategy.java
@@ -0,0 +1,154 @@
+package org.apache.hadoop.hive.ql.exec.tez.monitoring;
+
+import org.apache.hadoop.hive.common.log.InPlaceUpdate;
+import org.apache.hadoop.hive.common.log.ProgressMonitor;
+import org.apache.hadoop.hive.ql.log.PerfLogger;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.dag.api.client.Progress;
+
+import java.io.StringWriter;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+class RenderStrategy {
+
+  interface UpdateFunction {
+    void update(DAGStatus status, Map<String, Progress> vertexProgressMap);
+  }
+
+  private abstract static class BaseUpdateFunction implements UpdateFunction {
+    private static final int PRINT_INTERVAL = 3000;
+
+    final TezJobMonitor monitor;
+    private final PerfLogger perfLogger;
+
+    private long lastPrintTime = 0L;
+    private String lastReport = null;
+
+    BaseUpdateFunction(TezJobMonitor monitor) {
+      this.monitor = monitor;
+      perfLogger = SessionState.getPerfLogger();
+    }
+
+    @Override
+    public void update(DAGStatus status, Map<String, Progress> vertexProgressMap) {
+      renderProgress(monitor.progressMonitor(status, vertexProgressMap));
+      String report = getReport(vertexProgressMap);
+      if (showReport(report)) {
+        renderReport(report);
+        lastReport = report;
+        lastPrintTime = System.currentTimeMillis();
+      }
+    }
+
+    private boolean showReport(String report) {
+      return !report.equals(lastReport)
+          || System.currentTimeMillis() >= lastPrintTime + PRINT_INTERVAL;
+    }
+
+    private String getReport(Map<String, Progress> progressMap) {
+      StringWriter reportBuffer = new StringWriter();
+
+      SortedSet<String> keys = new TreeSet<String>(progressMap.keySet());
+      for (String s : keys) {
+        Progress progress = progressMap.get(s);
+        final int complete = progress.getSucceededTaskCount();
+        final int total = progress.getTotalTaskCount();
+        final int running = progress.getRunningTaskCount();
+        final int failed = progress.getFailedTaskAttemptCount();
+        if (total <= 0) {
+          reportBuffer.append(String.format("%s: -/-\t", s));
+        } else {
+          if (complete == total) {
+          /*
+           * We may have missed the start of the vertex due to the 3 seconds interval
+           */
+            if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) {
+              perfLogger.PerfLogBegin(TezJobMonitor.CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s);
+            }
+
+            perfLogger.PerfLogEnd(TezJobMonitor.CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s);
+          }
+          if (complete < total && (complete > 0 || running > 0 || failed > 0)) {
+
+            if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) {
+              perfLogger.PerfLogBegin(TezJobMonitor.CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s);
+            }
+
+          /* vertex is started, but not complete */
+            if (failed > 0) {
+              reportBuffer.append(
+                  String.format("%s: %d(+%d,-%d)/%d\t", s, complete, running, failed, total));
+            } else {
+              reportBuffer.append(String.format("%s: %d(+%d)/%d\t", s, complete, running, total));
+            }
+          } else {
+          /* vertex is waiting for input/slots or complete */
+            if (failed > 0) {
+            /* tasks finished but some failed */
+              reportBuffer.append(String.format("%s: %d(-%d)/%d\t", s, complete, failed, total));
+            } else {
+              reportBuffer.append(String.format("%s: %d/%d\t", s, complete, total));
+            }
+          }
+        }
+      }
+
+      return reportBuffer.toString();
+    }
+
+    abstract void renderProgress(ProgressMonitor progressMonitor);
+
+    abstract void renderReport(String report);
+  }
+
+  /**
+   * this adds the required progress update to the session state that is used by HS2 to send the
+   * same information to beeline client when requested.
+   */
+  static class LogToFileFunction extends BaseUpdateFunction {
+
+    LogToFileFunction(TezJobMonitor monitor) {
+      super(monitor);
+    }
+
+    @Override
+    public void renderProgress(ProgressMonitor progressMonitor) {
+      SessionState.get().updateProgressMonitor(progressMonitor);
+    }
+
+    @Override
+    public void renderReport(String report) {
+      monitor.console.printInfo(report);
+    }
+  }
+
+  /**
+   * This used when we want the progress update to printed in the same process typically used via
+   * hive-cli mode.
+   */
+  static class InPlaceUpdateFunction extends BaseUpdateFunction {
+    /**
+     * Have to use the same instance to render else the number lines printed earlier is lost and the
+     * screen will print the table again and again.
+     */
+    private final InPlaceUpdate inPlaceUpdate;
+
+    InPlaceUpdateFunction(TezJobMonitor monitor) {
+      super(monitor);
+      inPlaceUpdate = new InPlaceUpdate(SessionState.LogHelper.getInfoStream());
+    }
+
+    @Override
+    public void renderProgress(ProgressMonitor progressMonitor) {
+      inPlaceUpdate.render(progressMonitor);
+    }
+
+    @Override
+    public void renderReport(String report) {
+      monitor.console.logInfo(report);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e17a0409/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java
index c0a068d..f2f97f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/monitoring/TezJobMonitor.java
@@ -47,8 +47,6 @@ import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.SortedSet;
-import java.util.TreeSet;
 
 import static org.apache.tez.dag.api.client.DAGStatus.State.RUNNING;
 
@@ -59,24 +57,18 @@ import static org.apache.tez.dag.api.client.DAGStatus.State.RUNNING;
  */
 public class TezJobMonitor {
 
-  private static final String CLASS_NAME = TezJobMonitor.class.getName();
+  static final String CLASS_NAME = TezJobMonitor.class.getName();
   private static final int CHECK_INTERVAL = 200;
   private static final int MAX_RETRY_INTERVAL = 2500;
-  private static final int PRINT_INTERVAL = 3000;
 
   private final PerfLogger perfLogger = SessionState.getPerfLogger();
   private static final List<DAGClient> shutdownList;
   private final Map<String, BaseWork> workMap;
 
-  private transient LogHelper console;
+  transient LogHelper console;
 
-  private long lastPrintTime;
   private StringWriter diagnostics = new StringWriter();
 
-  interface UpdateFunction {
-    void update(DAGStatus status, Map<String, Progress> vertexProgressMap, String report);
-  }
-
   static {
     shutdownList = new LinkedList<>();
     ShutdownHookManager.addShutdownHook(new Runnable() {
@@ -102,12 +94,7 @@ public class TezJobMonitor {
   private final DAG dag;
   private final Context context;
   private long executionStartTime = 0;
-  private final UpdateFunction updateFunction;
-  /**
-   * Have to use the same instance to render else the number lines printed earlier is lost and the
-   * screen will print the table again and again.
-   */
-  private final InPlaceUpdate inPlaceUpdate;
+  private final RenderStrategy.UpdateFunction updateFunction;
 
   public TezJobMonitor(Map<String, BaseWork> workMap, final DAGClient dagClient, HiveConf conf, DAG dag,
                        Context ctx) {
@@ -117,29 +104,15 @@ public class TezJobMonitor {
     this.dag = dag;
     this.context = ctx;
     console = SessionState.getConsole();
-    inPlaceUpdate = new InPlaceUpdate(LogHelper.getInfoStream());
     updateFunction = updateFunction();
   }
 
-  private UpdateFunction updateFunction() {
-    UpdateFunction logToFileFunction = new UpdateFunction() {
-      @Override
-      public void update(DAGStatus status, Map<String, Progress> vertexProgressMap, String report) {
-        SessionState.get().updateProgressMonitor(progressMonitor(status, vertexProgressMap));
-        console.printInfo(report);
-      }
-    };
-    UpdateFunction inPlaceUpdateFunction = new UpdateFunction() {
-      @Override
-      public void update(DAGStatus status, Map<String, Progress> vertexProgressMap, String report) {
-        inPlaceUpdate.render(progressMonitor(status, vertexProgressMap));
-        console.logInfo(report);
-      }
-    };
+  private RenderStrategy.UpdateFunction updateFunction() {
     return InPlaceUpdate.canRenderInPlace(hiveConf)
         && !SessionState.getConsole().getIsSilent()
         && !SessionState.get().isHiveServerQuery()
-        ? inPlaceUpdateFunction : logToFileFunction;
+        ? new RenderStrategy.InPlaceUpdateFunction(this)
+        : new RenderStrategy.LogToFileFunction(this);
   }
 
   private boolean isProfilingEnabled() {
@@ -163,7 +136,6 @@ public class TezJobMonitor {
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_DAG);
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_SUBMIT_TO_RUNNING);
     DAGStatus.State lastState = null;
-    String lastReport = null;
     boolean running = false;
 
     while (true) {
@@ -195,13 +167,13 @@ public class TezJobMonitor {
                 this.executionStartTime = System.currentTimeMillis();
                 running = true;
               }
-              lastReport = updateStatus(status, vertexProgressMap, lastReport);
+              updateFunction.update(status, vertexProgressMap);
               break;
             case SUCCEEDED:
               if (!running) {
                 this.executionStartTime = monitorStartTime;
               }
-              lastReport = updateStatus(status, vertexProgressMap, lastReport);
+              updateFunction.update(status, vertexProgressMap);
               success = true;
               running = false;
               done = true;
@@ -210,7 +182,7 @@ public class TezJobMonitor {
               if (!running) {
                 this.executionStartTime = monitorStartTime;
               }
-              lastReport = updateStatus(status, vertexProgressMap, lastReport);
+              updateFunction.update(status, vertexProgressMap);
               console.printInfo("Status: Killed");
               running = false;
               done = true;
@@ -221,7 +193,7 @@ public class TezJobMonitor {
               if (!running) {
                 this.executionStartTime = monitorStartTime;
               }
-              lastReport = updateStatus(status, vertexProgressMap, lastReport);
+              updateFunction.update(status, vertexProgressMap);
               console.printError("Status: Failed");
               running = false;
               done = true;
@@ -323,71 +295,11 @@ public class TezJobMonitor {
     return (tezCounter == null) ? 0 : tezCounter.getValue();
   }
 
-  private String updateStatus(DAGStatus status, Map<String, Progress> vertexProgressMap,
-      String lastReport) {
-    String report = getReport(vertexProgressMap);
-    if (!report.equals(lastReport) || System.currentTimeMillis() >= lastPrintTime + PRINT_INTERVAL) {
-      updateFunction.update(status, vertexProgressMap, report);
-      lastPrintTime = System.currentTimeMillis();
-    }
-    return report;
-  }
-
-  private String getReport(Map<String, Progress> progressMap) {
-    StringBuilder reportBuffer = new StringBuilder();
-
-    SortedSet<String> keys = new TreeSet<String>(progressMap.keySet());
-    for (String s : keys) {
-      Progress progress = progressMap.get(s);
-      final int complete = progress.getSucceededTaskCount();
-      final int total = progress.getTotalTaskCount();
-      final int running = progress.getRunningTaskCount();
-      final int failed = progress.getFailedTaskAttemptCount();
-      if (total <= 0) {
-        reportBuffer.append(String.format("%s: -/-\t", s));
-      } else {
-        if (complete == total) {
-          /*
-           * We may have missed the start of the vertex due to the 3 seconds interval
-           */
-          if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) {
-            perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s);
-          }
-
-          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s);
-        }
-        if (complete < total && (complete > 0 || running > 0 || failed > 0)) {
-
-          if (!perfLogger.startTimeHasMethod(PerfLogger.TEZ_RUN_VERTEX + s)) {
-            perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_VERTEX + s);
-          }
-
-          /* vertex is started, but not complete */
-          if (failed > 0) {
-            reportBuffer.append(String.format("%s: %d(+%d,-%d)/%d\t", s, complete, running, failed, total));
-          } else {
-            reportBuffer.append(String.format("%s: %d(+%d)/%d\t", s, complete, running, total));
-          }
-        } else {
-          /* vertex is waiting for input/slots or complete */
-          if (failed > 0) {
-            /* tasks finished but some failed */
-            reportBuffer.append(String.format("%s: %d(-%d)/%d\t", s, complete, failed, total));
-          } else {
-            reportBuffer.append(String.format("%s: %d/%d\t", s, complete, total));
-          }
-        }
-      }
-    }
-
-    return reportBuffer.toString();
-  }
-
   public String getDiagnostics() {
     return diagnostics.toString();
   }
 
-  private ProgressMonitor progressMonitor(DAGStatus status, Map<String, Progress> progressMap) {
+  ProgressMonitor progressMonitor(DAGStatus status, Map<String, Progress> progressMap) {
     try {
       return new TezProgressMonitor(dagClient, status, workMap, progressMap, console,
           executionStartTime);


[25/50] [abbrv] hive git commit: HIVE-15992: LLAP: NPE in LlapTaskCommunicator.getCompletedLogsUrl for unsuccessful attempt (Rajesh Balamohan reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-15992: LLAP: NPE in LlapTaskCommunicator.getCompletedLogsUrl for unsuccessful attempt (Rajesh Balamohan 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/7fa8e37f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7fa8e37f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7fa8e37f

Branch: refs/heads/hive-14535
Commit: 7fa8e37fd13d9d6a4a4a5b2c72ce02d7c2d199ef
Parents: bda64ee
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Feb 21 10:24:28 2017 -0800
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Feb 21 10:24:28 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7fa8e37f/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
index 893b7d9..3aae7a4 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
@@ -541,7 +541,7 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
   @Override
   public String getInProgressLogsUrl(TezTaskAttemptID attemptID, NodeId containerNodeId) {
     String url = "";
-    if (timelineServerUri != null) {
+    if (timelineServerUri != null && containerNodeId != null) {
       LlapNodeId llapNodeId = LlapNodeId.getInstance(containerNodeId.getHost(), containerNodeId.getPort());
       BiMap<ContainerId, TezTaskAttemptID> biMap = entityTracker.getContainerAttemptMapForNode(llapNodeId);
       ContainerId containerId = biMap.inverse().get(attemptID);
@@ -559,7 +559,7 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
   @Override
   public String getCompletedLogsUrl(TezTaskAttemptID attemptID, NodeId containerNodeId) {
     String url = "";
-    if (timelineServerUri != null) {
+    if (timelineServerUri != null && containerNodeId != null) {
       LlapNodeId llapNodeId = LlapNodeId.getInstance(containerNodeId.getHost(), containerNodeId.getPort());
       BiMap<ContainerId, TezTaskAttemptID> biMap = entityTracker.getContainerAttemptMapForNode(llapNodeId);
       ContainerId containerId = biMap.inverse().get(attemptID);


[38/50] [abbrv] hive git commit: HIVE-16002 : Correlated IN subquery with aggregate asserts in sq_count_check UDF (Vineet Garg via Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-16002 : Correlated IN subquery with aggregate asserts in sq_count_check UDF (Vineet Garg via Ashutosh Chauhan)

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


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

Branch: refs/heads/hive-14535
Commit: 89310fee3ab7fa8cf4347850aaa03a57e10e78ba
Parents: ede8a55
Author: Vineet Garg <vg...@hortonworks.com>
Authored: Wed Feb 22 18:15:40 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Wed Feb 22 18:16:52 2017 -0800

----------------------------------------------------------------------
 .../ql/udf/generic/GenericUDFSQCountCheck.java  | 11 ++++--
 .../clientnegative/subquery_corr_in_agg.q       |  8 ++++
 .../clientnegative/subquery_corr_in_agg.q.out   | 39 ++++++++++++++++++++
 3 files changed, 54 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/89310fee/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSQCountCheck.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSQCountCheck.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSQCountCheck.java
index 89fa0de..f5d9f82 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSQCountCheck.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFSQCountCheck.java
@@ -61,11 +61,8 @@ public class GenericUDFSQCountCheck extends GenericUDF {
 
   @Override
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
-    Object valObject = arguments[0].get();
-    assert(valObject != null);
 
     Long val = getLongValue(arguments, 0, converters);
-    assert(val >= 0);
 
     switch (arguments.length){
       case 1: //Scalar queries, should expect value/count less than 1
@@ -75,7 +72,13 @@ public class GenericUDFSQCountCheck extends GenericUDF {
         }
         break;
       case 2:
-        if (val == 0) { // IN/NOT IN subqueries with aggregate
+        Object valObject = arguments[0].get();
+        if( valObject != null
+                && getLongValue(arguments, 0, converters) == 0){
+          throw new UDFArgumentException(
+                  " IN/NOT IN subquery with aggregate returning zero result. Currently this is not supported.");
+        }
+        else if(valObject == null) {
           throw new UDFArgumentException(
                   " IN/NOT IN subquery with aggregate returning zero result. Currently this is not supported.");
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/89310fee/ql/src/test/queries/clientnegative/subquery_corr_in_agg.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/subquery_corr_in_agg.q b/ql/src/test/queries/clientnegative/subquery_corr_in_agg.q
new file mode 100644
index 0000000..f677fba
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/subquery_corr_in_agg.q
@@ -0,0 +1,8 @@
+create table Part1 (PNum int, OrderOnHand int);
+insert into Part1 values (3,6),(10,1),(8,0);
+create table Supply (PNum int, Qty int);
+insert into Supply values (3,4),(3,2),(10,1);
+
+
+select pnum from Part1 p where OrderOnHand in
+                (select count(*) from Supply s where s.pnum = p.pnum);

http://git-wip-us.apache.org/repos/asf/hive/blob/89310fee/ql/src/test/results/clientnegative/subquery_corr_in_agg.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/subquery_corr_in_agg.q.out b/ql/src/test/results/clientnegative/subquery_corr_in_agg.q.out
new file mode 100644
index 0000000..36019cb
--- /dev/null
+++ b/ql/src/test/results/clientnegative/subquery_corr_in_agg.q.out
@@ -0,0 +1,39 @@
+PREHOOK: query: create table Part1 (PNum int, OrderOnHand int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@Part1
+POSTHOOK: query: create table Part1 (PNum int, OrderOnHand int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@Part1
+PREHOOK: query: insert into Part1 values (3,6),(10,1),(8,0)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part1
+POSTHOOK: query: insert into Part1 values (3,6),(10,1),(8,0)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part1
+POSTHOOK: Lineage: part1.orderonhand EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: part1.pnum EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: create table Supply (PNum int, Qty int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@Supply
+POSTHOOK: query: create table Supply (PNum int, Qty int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@Supply
+PREHOOK: query: insert into Supply values (3,4),(3,2),(10,1)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@supply
+POSTHOOK: query: insert into Supply values (3,4),(3,2),(10,1)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@supply
+POSTHOOK: Lineage: supply.pnum EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: supply.qty EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: select pnum from Part1 p where OrderOnHand in
+                (select count(*) from Supply s where s.pnum = p.pnum)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part1
+PREHOOK: Input: default@supply
+#### A masked pattern was here ####
+FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask


[09/50] [abbrv] hive git commit: HIVE-15915: Emit progress percentage in getting operation status (Jimmy Xiang, reviewed by Xuefu Zhang)

Posted by se...@apache.org.
HIVE-15915: Emit progress percentage in getting operation status (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/1677ed95
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1677ed95
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1677ed95

Branch: refs/heads/hive-14535
Commit: 1677ed956edb660c02b0d15253d486e2b0b626ba
Parents: bb4d8db
Author: Jimmy Xiang <jx...@apache.org>
Authored: Tue Feb 14 10:27:14 2017 -0800
Committer: Jimmy Xiang <jx...@apache.org>
Committed: Fri Feb 17 09:48:37 2017 -0800

----------------------------------------------------------------------
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |  6 +++-
 .../ql/exec/spark/status/SparkJobMonitor.java   | 12 +++++++
 .../hadoop/hive/ql/session/SessionState.java    | 36 +++++++++++++++++++-
 .../service/cli/thrift/ThriftCLIService.java    | 24 +++++++++++--
 4 files changed, 74 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1677ed95/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
index 41887d7..3c07197 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
@@ -92,6 +92,11 @@ public class HadoopJobExecHelper {
     reduceProgress = reduceProgress == 100 ? (int)Math.floor(rj.reduceProgress() * 100) : reduceProgress;
     task.taskCounters.put("CNTR_NAME_" + task.getId() + "_MAP_PROGRESS", Long.valueOf(mapProgress));
     task.taskCounters.put("CNTR_NAME_" + task.getId() + "_REDUCE_PROGRESS", Long.valueOf(reduceProgress));
+
+    if (SessionState.get() != null) {
+      final float progress = (rj.mapProgress() + rj.reduceProgress()) * 0.5f;
+      SessionState.get().updateProgressedPercentage(progress);
+    }
   }
 
   /**
@@ -196,7 +201,6 @@ public class HadoopJobExecHelper {
     }
   }
 
-  @SuppressWarnings("deprecation")
   public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg) {
     if (ctrs == null) {
       // hadoop might return null if it cannot locate the job.

http://git-wip-us.apache.org/repos/asf/hive/blob/1677ed95/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java
index cf0162d..0b224f2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java
@@ -179,6 +179,10 @@ abstract class SparkJobMonitor {
     String currentDate = dt.format(new Date());
     reportBuffer.append(currentDate + "\t");
 
+    // Num of total and completed tasks
+    int sumTotal = 0;
+    int sumComplete = 0;
+
     SortedSet<String> keys = new TreeSet<String>(progressMap.keySet());
     for (String s : keys) {
       SparkStageProgress progress = progressMap.get(s);
@@ -186,6 +190,9 @@ abstract class SparkJobMonitor {
       final int total = progress.getTotalTaskCount();
       final int running = progress.getRunningTaskCount();
       final int failed = progress.getFailedTaskCount();
+      sumTotal += total;
+      sumComplete += complete;
+
       String stageName = "Stage-" + s;
       if (total <= 0) {
         reportBuffer.append(String.format("%s: -/-\t", stageName));
@@ -230,6 +237,11 @@ abstract class SparkJobMonitor {
         }
       }
     }
+
+    if (SessionState.get() != null) {
+      final float progress = (sumTotal == 0) ? 1.0f : (float) sumComplete / (float) sumTotal;
+      SessionState.get().updateProgressedPercentage(progress);
+    }
     return reportBuffer.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/1677ed95/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 3e01e92..ba2c9c3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.common.log.ProgressMonitor;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.ObjectStore;
@@ -69,7 +70,6 @@ import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
 import org.apache.hadoop.hive.ql.lockmgr.LockException;
 import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
-import org.apache.hadoop.hive.common.log.ProgressMonitor;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
@@ -1748,6 +1748,40 @@ public class SessionState {
     return StringUtils.join(preReloadableAuxJars, ',');
   }
 
+  public void updateProgressedPercentage(final double percentage) {
+    this.progressMonitor = new ProgressMonitor() {
+      @Override
+      public List<String> headers() {
+        return null;
+      }
+
+      @Override
+      public List<List<String>> rows() {
+        return null;
+      }
+
+      @Override
+      public String footerSummary() {
+        return null;
+      }
+
+      @Override
+      public long startTime() {
+        return 0;
+      }
+
+      @Override
+      public String executionStatus() {
+        return null;
+      }
+
+      @Override
+      public double progressedPercentage() {
+        return percentage;
+      }
+    };
+  }
+
   public void updateProgressMonitor(ProgressMonitor progressMonitor) {
     this.progressMonitor = progressMonitor;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/1677ed95/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
index e09d9fe..211b33b 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
@@ -18,6 +18,8 @@
 
 package org.apache.hive.service.cli.thrift;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -27,9 +29,11 @@ import java.util.concurrent.TimeUnit;
 
 import javax.security.auth.login.LoginException;
 
+import org.apache.hadoop.hive.common.ServerUtils;
+import org.apache.hadoop.hive.common.log.ProgressMonitor;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.common.ServerUtils;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.service.AbstractService;
@@ -46,11 +50,13 @@ import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.JobProgressUpdate;
 import org.apache.hive.service.cli.OperationHandle;
 import org.apache.hive.service.cli.OperationStatus;
+import org.apache.hive.service.cli.OperationType;
 import org.apache.hive.service.cli.ProgressMonitorStatusMapper;
 import org.apache.hive.service.cli.RowSet;
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.cli.TableSchema;
 import org.apache.hive.service.cli.TezProgressMonitorStatusMapper;
+import org.apache.hive.service.cli.operation.Operation;
 import org.apache.hive.service.cli.session.SessionManager;
 import org.apache.hive.service.rpc.thrift.TCLIService;
 import org.apache.hive.service.rpc.thrift.TCancelDelegationTokenReq;
@@ -91,6 +97,7 @@ import org.apache.hive.service.rpc.thrift.TGetTablesReq;
 import org.apache.hive.service.rpc.thrift.TGetTablesResp;
 import org.apache.hive.service.rpc.thrift.TGetTypeInfoReq;
 import org.apache.hive.service.rpc.thrift.TGetTypeInfoResp;
+import org.apache.hive.service.rpc.thrift.TJobExecutionStatus;
 import org.apache.hive.service.rpc.thrift.TOpenSessionReq;
 import org.apache.hive.service.rpc.thrift.TOpenSessionResp;
 import org.apache.hive.service.rpc.thrift.TProgressUpdateResp;
@@ -431,6 +438,13 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
     return sessionHandle;
   }
 
+  private double getProgressedPercentage(OperationHandle opHandle) throws HiveSQLException {
+    checkArgument(OperationType.EXECUTE_STATEMENT.equals(opHandle.getOperationType()));
+    Operation operation = cliService.getSessionManager().getOperationManager().getOperation(opHandle);
+    SessionState state = operation.getParentSession().getSessionState();
+    ProgressMonitor monitor = state.getProgressMonitor();
+    return monitor == null ? 0.0 : monitor.progressedPercentage();
+  }
 
   private String getDelegationToken(String userName)
       throws HiveSQLException, LoginException, IOException {
@@ -646,11 +660,13 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
         mapper = new TezProgressMonitorStatusMapper();
       }
 
+      TJobExecutionStatus executionStatus =
+          mapper.forStatus(progressUpdate.status);
       resp.setProgressUpdateResponse(new TProgressUpdateResp(
           progressUpdate.headers(),
           progressUpdate.rows(),
           progressUpdate.progressedPercentage,
-          mapper.forStatus(progressUpdate.status),
+          executionStatus,
           progressUpdate.footerSummary,
           progressUpdate.startTimeMillis
       ));
@@ -659,6 +675,10 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
         resp.setErrorCode(opException.getErrorCode());
         resp.setErrorMessage(org.apache.hadoop.util.StringUtils.
             stringifyException(opException));
+      } else if (executionStatus == TJobExecutionStatus.NOT_AVAILABLE
+          && OperationType.EXECUTE_STATEMENT.equals(operationHandle.getOperationType())) {
+        resp.getProgressUpdateResponse().setProgressedPercentage(
+            getProgressedPercentage(operationHandle));
       }
       resp.setStatus(OK_STATUS);
     } catch (Exception e) {


[17/50] [abbrv] hive git commit: HIVE-15846 : Relocate more dependencies (e.g. org.apache.zookeeper) for JDBC uber jar (Tao Li via Thejas Nair)

Posted by se...@apache.org.
HIVE-15846 : Relocate more dependencies (e.g. org.apache.zookeeper) for JDBC uber jar (Tao Li 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/56f6c9dd
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/56f6c9dd
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/56f6c9dd

Branch: refs/heads/hive-14535
Commit: 56f6c9dd56e839a57dc93f7ae3c27866d62eb9fc
Parents: 1a6902c
Author: Tao Li <tl...@hortonworks.com>
Authored: Sat Feb 18 21:12:33 2017 -0800
Committer: Thejas M Nair <th...@hortonworks.com>
Committed: Sat Feb 18 21:12:38 2017 -0800

----------------------------------------------------------------------
 jdbc/pom.xml | 21 +++++++++++++++++----
 1 file changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/56f6c9dd/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index b68f207..25430c6 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -173,19 +173,18 @@
                     <exclude>META-INF/*.SF</exclude>
                     <exclude>META-INF/*.DSA</exclude>
                     <exclude>META-INF/*.RSA</exclude>
+                    <exclude>core-default.xml</exclude>
                   </excludes>
                 </filter>
               </filters>
               <artifactSet>
                 <excludes>
                   <exclude>org.apache.commons:commons-compress</exclude>
-                  <exclude>commons-configuration:commons-configuration</exclude>
                   <exclude>org.apache.hadoop:hadoop-yarn*</exclude>
                   <exclude>org.apache.hadoop:hadoop-mapreduce*</exclude>
                   <exclude>org.apache.hadoop:hadoop-hdfs</exclude>
                   <exclude>org.apache.hadoop:hadoop-client</exclude>
                   <exclude>org.apache.hadoop:hadoop-annotations</exclude>
-                  <exclude>org.apache.hadoop:hadoop-auth</exclude>
                   <exclude>org.apache.hive:hive-vector-code-gen</exclude>
                   <exclude>org.apache.ant:*</exclude>
                   <exclude>junit:*</exclude>
@@ -197,6 +196,7 @@
                   <exclude>org.eclipse.jetty.aggregate:*</exclude>
                   <exclude>org.tukaani:*</exclude>
                   <exclude>io.airlift:*</exclude>
+                  <exclude>io.dropwizard.metrics:*</exclude>
                   <exclude>org.apache.velocity:*</exclude>
                   <exclude>net.sf.jpam:*</exclude>
                   <exclude>org.apache.avro:*</exclude>
@@ -228,6 +228,7 @@
                   <exclude>com.sun.xml.bind:*</exclude>
                   <exclude>com.thoughtworks.paranamer:*</exclude>
                   <exclude>com.twitter:*</exclude>
+                  <exclude>com.zaxxer:*</exclude>
                   <exclude>io.netty:*</exclude>
                   <exclude>javax.activation:*</exclude>
                   <exclude>javax.inject:*</exclude>
@@ -236,8 +237,7 @@
                   <exclude>javax.servlet:*</exclude>
                   <exclude>javax.servlet.jsp:*</exclude>
                   <exclude>javax.transaction:*</exclude>
-                  <exclude>javax.xml.bind:*</exclude>
-                  <exclude>javax.xml.stream:*</exclude>
+                  <exclude>javax.xml.*:*</exclude>
                   <exclude>jline:*</exclude>
                   <exclude>joda-time:*</exclude>
                   <exclude>net.java.dev.jets3t:*</exclude>
@@ -246,6 +246,8 @@
                   <exclude>org.apache.directory.api:*</exclude>
                   <exclude>org.apache.directory.server:*</exclude>
                   <exclude>org.apache.geronimo.specs:*</exclude>
+                  <exclude>org.apache.htrace:*</exclude>
+                  <exclude>org.apache.slider:*</exclude>
                   <exclude>org.codehaus.jackson:*</exclude>
                   <exclude>org.codehaus.jettison:*</exclude>
                   <exclude>org.datanucleus:*</exclude>
@@ -253,8 +255,11 @@
                   <exclude>org.htrace:*</exclude>
                   <exclude>org.mortbay.jetty:*</exclude>
                   <exclude>org.xerial.snappy:*</exclude>
+                  <exclude>org.json:*</exclude>
                   <exclude>tomcat:*</exclude>
                   <exclude>xmlenc:*</exclude>
+                  <exclude>xerces:*</exclude>
+                  <exclude>xml-apis:*</exclude>
                 </excludes>
               </artifactSet>
               <relocations>
@@ -291,6 +296,14 @@
                   <shadedPattern>org.apache.hive.org.apache.hadoop</shadedPattern>
                 </relocation>
                 <relocation>
+                  <pattern>org.apache.zookeeper</pattern>
+                  <shadedPattern>org.apache.hive.org.apache.zookeeper</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.curator</pattern>
+                  <shadedPattern>org.apache.hive.org.apache.curator</shadedPattern>
+                </relocation>
+                <relocation>
                   <pattern>com.beust</pattern>
                   <shadedPattern>org.apache.hive.com.beust</shadedPattern>
                 </relocation>


[24/50] [abbrv] hive git commit: HIVE-15910 : Improvements in Hive Unit Test by using In-memory Derby DB (Sankar Hariappan via Thejas Nair, Wei Zheng)

Posted by se...@apache.org.
HIVE-15910 : Improvements in Hive Unit Test by using In-memory Derby DB (Sankar Hariappan via Thejas Nair, Wei Zheng)


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

Branch: refs/heads/hive-14535
Commit: bda64ee87c74a06b3cf19b08c41d67f192f22018
Parents: 016afe0
Author: Sankar Hariappan <ma...@gmail.com>
Authored: Mon Feb 20 22:51:29 2017 -0800
Committer: Thejas M Nair <th...@hortonworks.com>
Committed: Mon Feb 20 22:51:34 2017 -0800

----------------------------------------------------------------------
 data/conf/hive-site.xml                                  |  2 +-
 .../hadoop/hive/ql/txn/compactor/CompactorTest.java      | 11 +++++++----
 .../apache/hadoop/hive/ql/txn/compactor/TestCleaner.java |  6 ++++++
 .../hadoop/hive/ql/txn/compactor/TestInitiator.java      |  5 +++++
 .../apache/hadoop/hive/ql/txn/compactor/TestWorker.java  |  6 ++++++
 5 files changed, 25 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bda64ee8/data/conf/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/hive-site.xml b/data/conf/hive-site.xml
index 7a69711..62364fe 100644
--- a/data/conf/hive-site.xml
+++ b/data/conf/hive-site.xml
@@ -70,7 +70,7 @@
 
 <property>
   <name>javax.jdo.option.ConnectionURL</name>
-  <value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
+  <value>jdbc:derby:memory:${test.tmp.dir}/junit_metastore_db;create=true</value>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hive/blob/bda64ee8/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
index 2d1ecb5..bbed591 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.txn.compactor;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -66,6 +67,7 @@ import org.slf4j.LoggerFactory;
 import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -96,10 +98,11 @@ public abstract class CompactorTest {
     TxnDbUtil.cleanDb();
     ms = new HiveMetaStoreClient(conf);
     txnHandler = TxnUtils.getTxnStore(conf);
-    tmpdir = new File(System.getProperty("java.io.tmpdir") +
-        System.getProperty("file.separator") + "compactor_test_tables");
-    tmpdir.mkdir();
-    tmpdir.deleteOnExit();
+    tmpdir = new File (Files.createTempDirectory("compactor_test_table_").toString());
+  }
+
+  protected void compactorTestCleanup() throws IOException {
+    FileUtils.deleteDirectory(tmpdir);
   }
 
   protected void startInitiator() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/bda64ee8/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
index 44dd99b..0acf71b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.UnlockRequest;
 import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -517,4 +518,9 @@ public class TestCleaner extends CompactorTest {
   boolean useHive130DeltaDirName() {
     return false;
   }
+
+  @After
+  public void tearDown() throws Exception {
+    compactorTestCleanup();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bda64ee8/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
index a11fe86..f75a1be 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -752,4 +753,8 @@ public class TestInitiator extends CompactorTest {
     return false;
   }
 
+  @After
+  public void tearDown() throws Exception {
+    compactorTestCleanup();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/bda64ee8/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
index e85f49c..efd6ed8 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -977,4 +978,9 @@ public class TestWorker extends CompactorTest {
     Assert.assertEquals(1, compacts.size());
     Assert.assertTrue(TxnStore.SUCCEEDED_RESPONSE.equals(rsp.getCompacts().get(0).getState()));
   }
+
+  @After
+  public void tearDown() throws Exception {
+    compactorTestCleanup();
+  }
 }


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/ql/src/test/results/clientpositive/mm_all.q.out
----------------------------------------------------------------------
diff --cc ql/src/test/results/clientpositive/mm_all.q.out
index 4944d6c,0000000..4a9e4e5
mode 100644,000000..100644
--- a/ql/src/test/results/clientpositive/mm_all.q.out
+++ b/ql/src/test/results/clientpositive/mm_all.q.out
@@@ -1,3163 -1,0 +1,3163 @@@
 +PREHOOK: query: drop table intermediate
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table intermediate
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table intermediate(key int) partitioned by (p int) stored as orc
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@intermediate
 +POSTHOOK: query: create table intermediate(key int) partitioned by (p int) stored as orc
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@intermediate
 +PREHOOK: query: insert into table intermediate partition(p='455') select distinct key from src where key >= 0 order by key desc limit 2
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@src
 +PREHOOK: Output: default@intermediate@p=455
 +POSTHOOK: query: insert into table intermediate partition(p='455') select distinct key from src where key >= 0 order by key desc limit 2
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@src
 +POSTHOOK: Output: default@intermediate@p=455
 +POSTHOOK: Lineage: intermediate PARTITION(p=455).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 +PREHOOK: query: insert into table intermediate partition(p='456') select distinct key from src where key is not null order by key asc limit 2
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@src
 +PREHOOK: Output: default@intermediate@p=456
 +POSTHOOK: query: insert into table intermediate partition(p='456') select distinct key from src where key is not null order by key asc limit 2
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@src
 +POSTHOOK: Output: default@intermediate@p=456
 +POSTHOOK: Lineage: intermediate PARTITION(p=456).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 +PREHOOK: query: insert into table intermediate partition(p='457') select distinct key from src where key >= 100 order by key asc limit 2
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@src
 +PREHOOK: Output: default@intermediate@p=457
 +POSTHOOK: query: insert into table intermediate partition(p='457') select distinct key from src where key >= 100 order by key asc limit 2
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@src
 +POSTHOOK: Output: default@intermediate@p=457
 +POSTHOOK: Lineage: intermediate PARTITION(p=457).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 +PREHOOK: query: drop table part_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table part_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table part_mm(key int) partitioned by (key_mm int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@part_mm
 +POSTHOOK: query: create table part_mm(key int) partitioned by (key_mm int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +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: type: QUERY
 +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
 +  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: intermediate
 +            Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
 +            Select Operator
 +              expressions: key (type: int)
 +              outputColumnNames: _col0
 +              Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
 +              File Output Operator
 +                compressed: false
 +                Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
 +                table:
 +                    input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
 +                    output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 +                    serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
 +                    name: default.part_mm
 +
 +  Stage: Stage-7
 +    Conditional Operator
 +
 +  Stage: Stage-4
 +    Move Operator
 +      files:
 +          hdfs directory: true
 +#### A masked pattern was here ####
 +
 +  Stage: Stage-0
 +    Move Operator
 +      tables:
 +          partition:
 +            key_mm 455
 +          replace: false
 +          table:
 +              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
 +              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 +              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
 +              name: default.part_mm
 +          micromanaged table: true
 +
 +  Stage: Stage-2
 +    Stats-Aggr Operator
 +
 +  Stage: Stage-3
 +    Merge File Operator
 +      Map Operator Tree:
 +          ORC File Merge Operator
 +      merge level: stripe
 +      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
 +
 +  Stage: Stage-5
 +    Merge File Operator
 +      Map Operator Tree:
 +          ORC File Merge Operator
 +      merge level: stripe
 +      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
 +
 +  Stage: Stage-6
 +    Move Operator
 +      files:
 +          hdfs directory: true
 +#### A masked pattern was here ####
 +
 +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: 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@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: 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: 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@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: 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: 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@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: 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: 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: 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: 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
 +PREHOOK: Output: default@part_mm@key_mm=455
 +PREHOOK: Output: default@part_mm@key_mm=456
 +POSTHOOK: query: truncate table part_mm
 +POSTHOOK: type: TRUNCATETABLE
 +POSTHOOK: Output: default@part_mm@key_mm=455
 +POSTHOOK: Output: default@part_mm@key_mm=456
 +PREHOOK: query: 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: 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 ####
 +PREHOOK: query: drop table part_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@part_mm
 +PREHOOK: Output: default@part_mm
 +POSTHOOK: query: drop table part_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@part_mm
 +POSTHOOK: Output: default@part_mm
 +PREHOOK: query: drop table simple_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table simple_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table simple_mm(key int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@simple_mm
 +POSTHOOK: query: create table simple_mm(key int) stored as orc tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@simple_mm
 +PREHOOK: query: insert into table simple_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@simple_mm
 +POSTHOOK: query: insert into table simple_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@simple_mm
 +POSTHOOK: Lineage: simple_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert overwrite table simple_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@simple_mm
 +POSTHOOK: query: insert overwrite table simple_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@simple_mm
 +POSTHOOK: Lineage: simple_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from simple_mm order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_mm order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_mm
 +#### A masked pattern was here ####
 +0
 +10
 +97
 +98
 +100
 +103
 +PREHOOK: query: insert into table simple_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@simple_mm
 +POSTHOOK: query: insert into table simple_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@simple_mm
 +POSTHOOK: Lineage: simple_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from simple_mm order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_mm order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_mm
 +#### A masked pattern was here ####
 +0
 +0
 +10
 +10
 +97
 +97
 +98
 +98
 +100
 +100
 +103
 +103
 +PREHOOK: query: truncate table simple_mm
 +PREHOOK: type: TRUNCATETABLE
 +PREHOOK: Output: default@simple_mm
 +POSTHOOK: query: truncate table simple_mm
 +POSTHOOK: type: TRUNCATETABLE
 +POSTHOOK: Output: default@simple_mm
 +PREHOOK: query: select * from simple_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@simple_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from simple_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@simple_mm
 +#### A masked pattern was here ####
 +PREHOOK: query: drop table simple_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@simple_mm
 +PREHOOK: Output: default@simple_mm
 +POSTHOOK: query: drop table simple_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@simple_mm
 +POSTHOOK: Output: default@simple_mm
 +PREHOOK: query: drop table dp_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table dp_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table dp_mm (key int) partitioned by (key1 string, key2 int) stored as orc
 +  tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@dp_mm
 +POSTHOOK: query: create table dp_mm (key int) partitioned by (key1 string, key2 int) stored as orc
 +  tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@dp_mm
 +PREHOOK: query: insert into table dp_mm partition (key1='123', key2) select key, 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@dp_mm@key1=123
 +POSTHOOK: query: insert into table dp_mm partition (key1='123', key2) select key, 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@dp_mm@key1=123/key2=0
 +POSTHOOK: Output: default@dp_mm@key1=123/key2=10
 +POSTHOOK: Output: default@dp_mm@key1=123/key2=100
 +POSTHOOK: Output: default@dp_mm@key1=123/key2=103
 +POSTHOOK: Output: default@dp_mm@key1=123/key2=97
 +POSTHOOK: Output: default@dp_mm@key1=123/key2=98
 +POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=0).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=100).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=103).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=10).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=97).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=98).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from dp_mm order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@dp_mm
 +PREHOOK: Input: default@dp_mm@key1=123/key2=0
 +PREHOOK: Input: default@dp_mm@key1=123/key2=10
 +PREHOOK: Input: default@dp_mm@key1=123/key2=100
 +PREHOOK: Input: default@dp_mm@key1=123/key2=103
 +PREHOOK: Input: default@dp_mm@key1=123/key2=97
 +PREHOOK: Input: default@dp_mm@key1=123/key2=98
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from dp_mm order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@dp_mm
 +POSTHOOK: Input: default@dp_mm@key1=123/key2=0
 +POSTHOOK: Input: default@dp_mm@key1=123/key2=10
 +POSTHOOK: Input: default@dp_mm@key1=123/key2=100
 +POSTHOOK: Input: default@dp_mm@key1=123/key2=103
 +POSTHOOK: Input: default@dp_mm@key1=123/key2=97
 +POSTHOOK: Input: default@dp_mm@key1=123/key2=98
 +#### A masked pattern was here ####
 +0	123	0
 +10	123	10
 +97	123	97
 +98	123	98
 +100	123	100
 +103	123	103
 +PREHOOK: query: drop table dp_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@dp_mm
 +PREHOOK: Output: default@dp_mm
 +POSTHOOK: query: drop table dp_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@dp_mm
 +POSTHOOK: Output: default@dp_mm
 +PREHOOK: query: create table union_mm(id int)  tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@union_mm
 +POSTHOOK: query: create table union_mm(id int)  tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@union_mm
 +PREHOOK: query: insert into table union_mm 
 +select temps.p from (
 +select key as p from intermediate 
 +union all 
 +select key + 1 as p from intermediate ) temps
 +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@union_mm
 +POSTHOOK: query: insert into table union_mm 
 +select temps.p from (
 +select key as p from intermediate 
 +union all 
 +select key + 1 as p from intermediate ) temps
 +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@union_mm
 +POSTHOOK: Lineage: union_mm.id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from union_mm order by id
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@union_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from union_mm order by id
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@union_mm
 +#### A masked pattern was here ####
 +0
 +1
 +10
 +11
 +97
 +98
 +98
 +99
 +100
 +101
 +103
 +104
 +PREHOOK: query: insert into table union_mm 
 +select p from
 +(
 +select key + 1 as p from intermediate
 +union all
 +select key from intermediate
 +) tab group by p
 +union all
 +select key + 2 as p 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@union_mm
 +POSTHOOK: query: insert into table union_mm 
 +select p from
 +(
 +select key + 1 as p from intermediate
 +union all
 +select key from intermediate
 +) tab group by p
 +union all
 +select key + 2 as p 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@union_mm
 +POSTHOOK: Lineage: union_mm.id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from union_mm order by id
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@union_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from union_mm order by id
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@union_mm
 +#### A masked pattern was here ####
 +0
 +0
 +1
 +1
 +2
 +10
 +10
 +11
 +11
 +12
 +97
 +97
 +98
 +98
 +98
 +99
 +99
 +99
 +100
 +100
 +100
 +101
 +101
 +102
 +103
 +103
 +104
 +104
 +105
 +PREHOOK: query: insert into table union_mm
 +SELECT p FROM
 +(
 +  SELECT key + 1 as p FROM intermediate
 +  UNION ALL
 +  SELECT key as p FROM ( 
 +    SELECT distinct key FROM (
 +      SELECT key FROM (
 +        SELECT key + 2 as key FROM intermediate
 +        UNION ALL
 +        SELECT key FROM intermediate
 +      )t1 
 +    group by key)t2
 +  )t3
 +)t4
 +group by p
 +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@union_mm
 +POSTHOOK: query: insert into table union_mm
 +SELECT p FROM
 +(
 +  SELECT key + 1 as p FROM intermediate
 +  UNION ALL
 +  SELECT key as p FROM ( 
 +    SELECT distinct key FROM (
 +      SELECT key FROM (
 +        SELECT key + 2 as key FROM intermediate
 +        UNION ALL
 +        SELECT key FROM intermediate
 +      )t1 
 +    group by key)t2
 +  )t3
 +)t4
 +group by p
 +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@union_mm
 +POSTHOOK: Lineage: union_mm.id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from union_mm order by id
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@union_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from union_mm order by id
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@union_mm
 +#### A masked pattern was here ####
 +0
 +0
 +0
 +1
 +1
 +1
 +2
 +2
 +10
 +10
 +10
 +11
 +11
 +11
 +12
 +12
 +97
 +97
 +97
 +98
 +98
 +98
 +98
 +99
 +99
 +99
 +99
 +100
 +100
 +100
 +100
 +101
 +101
 +101
 +102
 +102
 +103
 +103
 +103
 +104
 +104
 +104
 +105
 +105
 +PREHOOK: query: drop table union_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@union_mm
 +PREHOOK: Output: default@union_mm
 +POSTHOOK: query: drop table union_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@union_mm
 +POSTHOOK: Output: default@union_mm
 +PREHOOK: query: create table partunion_mm(id int) partitioned by (key int) tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@partunion_mm
 +POSTHOOK: query: create table partunion_mm(id int) partitioned by (key int) tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@partunion_mm
 +PREHOOK: query: insert into table partunion_mm partition(key)
 +select temps.* from (
 +select key as p, key from intermediate 
 +union all 
 +select key + 1 as p, key + 1 from intermediate ) temps
 +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@partunion_mm
 +POSTHOOK: query: insert into table partunion_mm partition(key)
 +select temps.* from (
 +select key as p, key from intermediate 
 +union all 
 +select key + 1 as p, key + 1 from intermediate ) temps
 +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@partunion_mm@key=0
 +POSTHOOK: Output: default@partunion_mm@key=1
 +POSTHOOK: Output: default@partunion_mm@key=10
 +POSTHOOK: Output: default@partunion_mm@key=100
 +POSTHOOK: Output: default@partunion_mm@key=101
 +POSTHOOK: Output: default@partunion_mm@key=103
 +POSTHOOK: Output: default@partunion_mm@key=104
 +POSTHOOK: Output: default@partunion_mm@key=11
 +POSTHOOK: Output: default@partunion_mm@key=97
 +POSTHOOK: Output: default@partunion_mm@key=98
 +POSTHOOK: Output: default@partunion_mm@key=99
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=0).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=100).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=101).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=103).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=104).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=10).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=11).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=1).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=97).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=98).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: partunion_mm PARTITION(key=99).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from partunion_mm order by id
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@partunion_mm
 +PREHOOK: Input: default@partunion_mm@key=0
 +PREHOOK: Input: default@partunion_mm@key=1
 +PREHOOK: Input: default@partunion_mm@key=10
 +PREHOOK: Input: default@partunion_mm@key=100
 +PREHOOK: Input: default@partunion_mm@key=101
 +PREHOOK: Input: default@partunion_mm@key=103
 +PREHOOK: Input: default@partunion_mm@key=104
 +PREHOOK: Input: default@partunion_mm@key=11
 +PREHOOK: Input: default@partunion_mm@key=97
 +PREHOOK: Input: default@partunion_mm@key=98
 +PREHOOK: Input: default@partunion_mm@key=99
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from partunion_mm order by id
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@partunion_mm
 +POSTHOOK: Input: default@partunion_mm@key=0
 +POSTHOOK: Input: default@partunion_mm@key=1
 +POSTHOOK: Input: default@partunion_mm@key=10
 +POSTHOOK: Input: default@partunion_mm@key=100
 +POSTHOOK: Input: default@partunion_mm@key=101
 +POSTHOOK: Input: default@partunion_mm@key=103
 +POSTHOOK: Input: default@partunion_mm@key=104
 +POSTHOOK: Input: default@partunion_mm@key=11
 +POSTHOOK: Input: default@partunion_mm@key=97
 +POSTHOOK: Input: default@partunion_mm@key=98
 +POSTHOOK: Input: default@partunion_mm@key=99
 +#### A masked pattern was here ####
 +0	0
 +1	1
 +10	10
 +11	11
 +97	97
 +98	98
 +98	98
 +99	99
 +100	100
 +101	101
 +103	103
 +104	104
 +PREHOOK: query: drop table partunion_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@partunion_mm
 +PREHOOK: Output: default@partunion_mm
 +POSTHOOK: query: drop table partunion_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@partunion_mm
 +POSTHOOK: Output: default@partunion_mm
 +PREHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
 + stored as directories tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@skew_mm
 +POSTHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
 + stored as directories tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@skew_mm
 +PREHOOK: query: insert into table skew_mm 
 +select key, key, 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@skew_mm
 +POSTHOOK: query: insert into table skew_mm 
 +select key, key, 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@skew_mm
 +POSTHOOK: Lineage: skew_mm.k1 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_mm.k2 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_mm.k4 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from skew_mm order by k2
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@skew_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from skew_mm order by k2
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@skew_mm
 +#### A masked pattern was here ####
 +0	0	0
 +10	10	10
 +97	97	97
 +98	98	98
 +100	100	100
 +103	103	103
 +PREHOOK: query: drop table skew_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@skew_mm
 +PREHOOK: Output: default@skew_mm
 +POSTHOOK: query: drop table skew_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@skew_mm
 +POSTHOOK: Output: default@skew_mm
 +PREHOOK: query: create table skew_dp_union_mm(k1 int, k2 int, k4 int) partitioned by (k3 int) 
 +skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3)) stored as directories tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@skew_dp_union_mm
 +POSTHOOK: query: create table skew_dp_union_mm(k1 int, k2 int, k4 int) partitioned by (k3 int) 
 +skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3)) stored as directories tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@skew_dp_union_mm
 +PREHOOK: query: insert into table skew_dp_union_mm partition (k3)
 +select key as i, key as j, key as k, key as l from intermediate
 +union all 
 +select key +1 as i, key +2 as j, key +3 as k, key +4 as l 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@skew_dp_union_mm
 +POSTHOOK: query: insert into table skew_dp_union_mm partition (k3)
 +select key as i, key as j, key as k, key as l from intermediate
 +union all 
 +select key +1 as i, key +2 as j, key +3 as k, key +4 as l 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@skew_dp_union_mm@k3=0
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=10
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=100
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=101
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=102
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=103
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=104
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=107
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=14
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=4
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=97
 +POSTHOOK: Output: default@skew_dp_union_mm@k3=98
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=0).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=0).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=0).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=100).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=100).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=100).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=101).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=101).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=101).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=102).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=102).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=102).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=103).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=103).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=103).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=104).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=104).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=104).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=107).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=107).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=107).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=10).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=10).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=10).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=14).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=14).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=14).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=4).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=4).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=4).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=97).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=97).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=97).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=98).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=98).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=98).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from skew_dp_union_mm order by k2
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@skew_dp_union_mm
 +PREHOOK: Input: default@skew_dp_union_mm@k3=0
 +PREHOOK: Input: default@skew_dp_union_mm@k3=10
 +PREHOOK: Input: default@skew_dp_union_mm@k3=100
 +PREHOOK: Input: default@skew_dp_union_mm@k3=101
 +PREHOOK: Input: default@skew_dp_union_mm@k3=102
 +PREHOOK: Input: default@skew_dp_union_mm@k3=103
 +PREHOOK: Input: default@skew_dp_union_mm@k3=104
 +PREHOOK: Input: default@skew_dp_union_mm@k3=107
 +PREHOOK: Input: default@skew_dp_union_mm@k3=14
 +PREHOOK: Input: default@skew_dp_union_mm@k3=4
 +PREHOOK: Input: default@skew_dp_union_mm@k3=97
 +PREHOOK: Input: default@skew_dp_union_mm@k3=98
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from skew_dp_union_mm order by k2
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@skew_dp_union_mm
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=0
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=10
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=100
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=101
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=102
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=103
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=104
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=107
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=14
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=4
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=97
 +POSTHOOK: Input: default@skew_dp_union_mm@k3=98
 +#### A masked pattern was here ####
 +0	0	0	0
 +1	2	3	4
 +10	10	10	10
 +11	12	13	14
 +97	97	97	97
 +98	98	98	98
 +98	99	100	101
 +99	100	101	102
 +100	100	100	100
 +101	102	103	104
 +103	103	103	103
 +104	105	106	107
 +PREHOOK: query: drop table skew_dp_union_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@skew_dp_union_mm
 +PREHOOK: Output: default@skew_dp_union_mm
 +POSTHOOK: query: drop table skew_dp_union_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@skew_dp_union_mm
 +POSTHOOK: Output: default@skew_dp_union_mm
 +PREHOOK: query: create table merge0_mm (id int) stored as orc tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@merge0_mm
 +POSTHOOK: query: create table merge0_mm (id int) stored as orc tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@merge0_mm
 +PREHOOK: query: insert into table merge0_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@merge0_mm
 +POSTHOOK: query: insert into table merge0_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@merge0_mm
 +POSTHOOK: Lineage: merge0_mm.id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from merge0_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@merge0_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from merge0_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@merge0_mm
 +#### A masked pattern was here ####
 +98
 +97
 +0
 +10
 +100
 +103
 +PREHOOK: query: insert into table merge0_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@merge0_mm
 +POSTHOOK: query: insert into table merge0_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@merge0_mm
 +POSTHOOK: Lineage: merge0_mm.id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from merge0_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@merge0_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from merge0_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@merge0_mm
 +#### A masked pattern was here ####
 +98
 +97
 +0
 +10
 +100
 +103
 +98
 +97
 +0
 +10
 +100
 +103
 +PREHOOK: query: drop table merge0_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@merge0_mm
 +PREHOOK: Output: default@merge0_mm
 +POSTHOOK: query: drop table merge0_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@merge0_mm
 +POSTHOOK: Output: default@merge0_mm
 +PREHOOK: query: create table merge2_mm (id int) tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@merge2_mm
 +POSTHOOK: query: create table merge2_mm (id int) tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@merge2_mm
 +PREHOOK: query: insert into table merge2_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@merge2_mm
 +POSTHOOK: query: insert into table merge2_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@merge2_mm
 +POSTHOOK: Lineage: merge2_mm.id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from merge2_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@merge2_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from merge2_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@merge2_mm
 +#### A masked pattern was here ####
 +98
 +97
 +0
 +10
 +100
 +103
 +PREHOOK: query: insert into table merge2_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@merge2_mm
 +POSTHOOK: query: insert into table merge2_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@merge2_mm
 +POSTHOOK: Lineage: merge2_mm.id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from merge2_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@merge2_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from merge2_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@merge2_mm
 +#### A masked pattern was here ####
 +98
 +97
 +0
 +10
 +100
 +103
 +98
 +97
 +0
 +10
 +100
 +103
 +PREHOOK: query: drop table merge2_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@merge2_mm
 +PREHOOK: Output: default@merge2_mm
 +POSTHOOK: query: drop table merge2_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@merge2_mm
 +POSTHOOK: Output: default@merge2_mm
 +PREHOOK: query: create table merge1_mm (id int) partitioned by (key int) stored as orc tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@merge1_mm
 +POSTHOOK: query: create table merge1_mm (id int) partitioned by (key int) stored as orc tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@merge1_mm
 +PREHOOK: query: insert into table merge1_mm partition (key) select key, 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@merge1_mm
 +POSTHOOK: query: insert into table merge1_mm partition (key) select key, 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@merge1_mm@key=0
 +POSTHOOK: Output: default@merge1_mm@key=10
 +POSTHOOK: Output: default@merge1_mm@key=100
 +POSTHOOK: Output: default@merge1_mm@key=103
 +POSTHOOK: Output: default@merge1_mm@key=97
 +POSTHOOK: Output: default@merge1_mm@key=98
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=0).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=100).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=103).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=10).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=97).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=98).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from merge1_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@merge1_mm
 +PREHOOK: Input: default@merge1_mm@key=0
 +PREHOOK: Input: default@merge1_mm@key=10
 +PREHOOK: Input: default@merge1_mm@key=100
 +PREHOOK: Input: default@merge1_mm@key=103
 +PREHOOK: Input: default@merge1_mm@key=97
 +PREHOOK: Input: default@merge1_mm@key=98
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from merge1_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@merge1_mm
 +POSTHOOK: Input: default@merge1_mm@key=0
 +POSTHOOK: Input: default@merge1_mm@key=10
 +POSTHOOK: Input: default@merge1_mm@key=100
 +POSTHOOK: Input: default@merge1_mm@key=103
 +POSTHOOK: Input: default@merge1_mm@key=97
 +POSTHOOK: Input: default@merge1_mm@key=98
 +#### A masked pattern was here ####
- 97	97
++100	100
 +103	103
 +98	98
- 100	100
++97	97
 +10	10
 +0	0
 +PREHOOK: query: insert into table merge1_mm partition (key) select key, 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@merge1_mm
 +POSTHOOK: query: insert into table merge1_mm partition (key) select key, 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@merge1_mm@key=0
 +POSTHOOK: Output: default@merge1_mm@key=10
 +POSTHOOK: Output: default@merge1_mm@key=100
 +POSTHOOK: Output: default@merge1_mm@key=103
 +POSTHOOK: Output: default@merge1_mm@key=97
 +POSTHOOK: Output: default@merge1_mm@key=98
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=0).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=100).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=103).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=10).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=97).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: merge1_mm PARTITION(key=98).id SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from merge1_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@merge1_mm
 +PREHOOK: Input: default@merge1_mm@key=0
 +PREHOOK: Input: default@merge1_mm@key=10
 +PREHOOK: Input: default@merge1_mm@key=100
 +PREHOOK: Input: default@merge1_mm@key=103
 +PREHOOK: Input: default@merge1_mm@key=97
 +PREHOOK: Input: default@merge1_mm@key=98
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from merge1_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@merge1_mm
 +POSTHOOK: Input: default@merge1_mm@key=0
 +POSTHOOK: Input: default@merge1_mm@key=10
 +POSTHOOK: Input: default@merge1_mm@key=100
 +POSTHOOK: Input: default@merge1_mm@key=103
 +POSTHOOK: Input: default@merge1_mm@key=97
 +POSTHOOK: Input: default@merge1_mm@key=98
 +#### A masked pattern was here ####
- 100	100
- 97	97
 +103	103
++100	100
 +103	103
++97	97
 +100	100
 +97	97
 +98	98
 +98	98
- 0	0
 +10	10
 +0	0
 +10	10
++0	0
 +PREHOOK: query: drop table merge1_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@merge1_mm
 +PREHOOK: Output: default@merge1_mm
 +POSTHOOK: query: drop table merge1_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@merge1_mm
 +POSTHOOK: Output: default@merge1_mm
 +PREHOOK: query: drop table ctas0_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table ctas0_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table ctas0_mm tblproperties ("transactional"="true", "transactional_properties"="insert_only") as select * from intermediate
 +PREHOOK: type: CREATETABLE_AS_SELECT
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@ctas0_mm
 +POSTHOOK: query: create table ctas0_mm tblproperties ("transactional"="true", "transactional_properties"="insert_only") as select * from intermediate
 +POSTHOOK: type: CREATETABLE_AS_SELECT
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@ctas0_mm
 +POSTHOOK: Lineage: ctas0_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: ctas0_mm.p SIMPLE [(intermediate)intermediate.FieldSchema(name:p, type:int, comment:null), ]
 +PREHOOK: query: select * from ctas0_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@ctas0_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from ctas0_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@ctas0_mm
 +#### A masked pattern was here ####
 +98	455
 +97	455
 +0	456
 +10	456
 +100	457
 +103	457
 +PREHOOK: query: drop table ctas0_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@ctas0_mm
 +PREHOOK: Output: default@ctas0_mm
 +POSTHOOK: query: drop table ctas0_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@ctas0_mm
 +POSTHOOK: Output: default@ctas0_mm
 +PREHOOK: query: drop table ctas1_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table ctas1_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table ctas1_mm tblproperties ("transactional"="true", "transactional_properties"="insert_only") as
 +  select * from intermediate union all select * from intermediate
 +PREHOOK: type: CREATETABLE_AS_SELECT
 +PREHOOK: Input: default@intermediate
 +PREHOOK: Input: default@intermediate@p=455
 +PREHOOK: Input: default@intermediate@p=456
 +PREHOOK: Input: default@intermediate@p=457
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@ctas1_mm
 +POSTHOOK: query: create table ctas1_mm tblproperties ("transactional"="true", "transactional_properties"="insert_only") as
 +  select * from intermediate union all select * from intermediate
 +POSTHOOK: type: CREATETABLE_AS_SELECT
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POSTHOOK: Input: default@intermediate@p=457
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@ctas1_mm
 +POSTHOOK: Lineage: ctas1_mm.key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: ctas1_mm.p EXPRESSION [(intermediate)intermediate.FieldSchema(name:p, type:int, comment:null), ]
 +PREHOOK: query: select * from ctas1_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@ctas1_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from ctas1_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@ctas1_mm
 +#### A masked pattern was here ####
 +98	455
 +98	455
 +97	455
 +97	455
 +0	456
 +0	456
 +10	456
 +10	456
 +100	457
 +100	457
 +103	457
 +103	457
 +PREHOOK: query: drop table ctas1_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@ctas1_mm
 +PREHOOK: Output: default@ctas1_mm
 +POSTHOOK: query: drop table ctas1_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@ctas1_mm
 +POSTHOOK: Output: default@ctas1_mm
 +PREHOOK: query: drop table iow0_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table iow0_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table iow0_mm(key int) tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@iow0_mm
 +POSTHOOK: query: create table iow0_mm(key int) tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@iow0_mm
 +PREHOOK: query: insert overwrite table iow0_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@iow0_mm
 +POSTHOOK: query: insert overwrite table iow0_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@iow0_mm
 +POSTHOOK: Lineage: iow0_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table iow0_mm select key + 1 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@iow0_mm
 +POSTHOOK: query: insert into table iow0_mm select key + 1 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@iow0_mm
 +POSTHOOK: Lineage: iow0_mm.key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from iow0_mm order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@iow0_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from iow0_mm order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@iow0_mm
 +#### A masked pattern was here ####
 +0
 +1
 +10
 +11
 +97
 +98
 +98
 +99
 +100
 +101
 +103
 +104
 +PREHOOK: query: insert overwrite table iow0_mm select key + 2 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@iow0_mm
 +POSTHOOK: query: insert overwrite table iow0_mm select key + 2 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@iow0_mm
 +POSTHOOK: Lineage: iow0_mm.key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from iow0_mm order by key
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@iow0_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from iow0_mm order by key
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@iow0_mm
 +#### A masked pattern was here ####
 +2
 +12
 +99
 +100
 +102
 +105
 +PREHOOK: query: drop table iow0_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@iow0_mm
 +PREHOOK: Output: default@iow0_mm
 +POSTHOOK: query: drop table iow0_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@iow0_mm
 +POSTHOOK: Output: default@iow0_mm
 +PREHOOK: query: drop table iow1_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table iow1_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table iow1_mm(key int) partitioned by (key2 int)  tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@iow1_mm
 +POSTHOOK: query: create table iow1_mm(key int) partitioned by (key2 int)  tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@iow1_mm
 +PREHOOK: query: insert overwrite table iow1_mm partition (key2)
 +select key as k1, key from intermediate union all select key as k1, 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@iow1_mm
 +POSTHOOK: query: insert overwrite table iow1_mm partition (key2)
 +select key as k1, key from intermediate union all select key as k1, 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@iow1_mm@key2=0
 +POSTHOOK: Output: default@iow1_mm@key2=10
 +POSTHOOK: Output: default@iow1_mm@key2=100
 +POSTHOOK: Output: default@iow1_mm@key2=103
 +POSTHOOK: Output: default@iow1_mm@key2=97
 +POSTHOOK: Output: default@iow1_mm@key2=98
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=0).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=100).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=103).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=10).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=97).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=98).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: insert into table iow1_mm partition (key2)
 +select key + 1 as k1, key from intermediate union all select key as k1, 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@iow1_mm
 +POSTHOOK: query: insert into table iow1_mm partition (key2)
 +select key + 1 as k1, key from intermediate union all select key as k1, 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@iow1_mm@key2=0
 +POSTHOOK: Output: default@iow1_mm@key2=10
 +POSTHOOK: Output: default@iow1_mm@key2=100
 +POSTHOOK: Output: default@iow1_mm@key2=103
 +POSTHOOK: Output: default@iow1_mm@key2=97
 +POSTHOOK: Output: default@iow1_mm@key2=98
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=0).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=100).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=103).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=10).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=97).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=98).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from iow1_mm order by key, key2
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@iow1_mm
 +PREHOOK: Input: default@iow1_mm@key2=0
 +PREHOOK: Input: default@iow1_mm@key2=10
 +PREHOOK: Input: default@iow1_mm@key2=100
 +PREHOOK: Input: default@iow1_mm@key2=103
 +PREHOOK: Input: default@iow1_mm@key2=97
 +PREHOOK: Input: default@iow1_mm@key2=98
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from iow1_mm order by key, key2
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@iow1_mm
 +POSTHOOK: Input: default@iow1_mm@key2=0
 +POSTHOOK: Input: default@iow1_mm@key2=10
 +POSTHOOK: Input: default@iow1_mm@key2=100
 +POSTHOOK: Input: default@iow1_mm@key2=103
 +POSTHOOK: Input: default@iow1_mm@key2=97
 +POSTHOOK: Input: default@iow1_mm@key2=98
 +#### A masked pattern was here ####
 +0	0
 +0	0
 +0	0
 +1	0
 +10	10
 +10	10
 +10	10
 +11	10
 +97	97
 +97	97
 +97	97
 +98	97
 +98	98
 +98	98
 +98	98
 +99	98
 +100	100
 +100	100
 +100	100
 +101	100
 +103	103
 +103	103
 +103	103
 +104	103
 +PREHOOK: query: insert overwrite table iow1_mm partition (key2)
 +select key + 3 as k1, key from intermediate union all select key + 4 as k1, 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@iow1_mm
 +POSTHOOK: query: insert overwrite table iow1_mm partition (key2)
 +select key + 3 as k1, key from intermediate union all select key + 4 as k1, 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@iow1_mm@key2=0
 +POSTHOOK: Output: default@iow1_mm@key2=10
 +POSTHOOK: Output: default@iow1_mm@key2=100
 +POSTHOOK: Output: default@iow1_mm@key2=103
 +POSTHOOK: Output: default@iow1_mm@key2=97
 +POSTHOOK: Output: default@iow1_mm@key2=98
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=0).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=100).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=103).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=10).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=97).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=98).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from iow1_mm order by key, key2
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@iow1_mm
 +PREHOOK: Input: default@iow1_mm@key2=0
 +PREHOOK: Input: default@iow1_mm@key2=10
 +PREHOOK: Input: default@iow1_mm@key2=100
 +PREHOOK: Input: default@iow1_mm@key2=103
 +PREHOOK: Input: default@iow1_mm@key2=97
 +PREHOOK: Input: default@iow1_mm@key2=98
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from iow1_mm order by key, key2
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@iow1_mm
 +POSTHOOK: Input: default@iow1_mm@key2=0
 +POSTHOOK: Input: default@iow1_mm@key2=10
 +POSTHOOK: Input: default@iow1_mm@key2=100
 +POSTHOOK: Input: default@iow1_mm@key2=103
 +POSTHOOK: Input: default@iow1_mm@key2=97
 +POSTHOOK: Input: default@iow1_mm@key2=98
 +#### A masked pattern was here ####
 +3	0
 +4	0
 +13	10
 +14	10
 +100	97
 +101	97
 +101	98
 +102	98
 +103	100
 +104	100
 +106	103
 +107	103
 +PREHOOK: query: insert overwrite table iow1_mm partition (key2)
 +select key + 3 as k1, key + 3 from intermediate union all select key + 2 as k1, key + 2 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@iow1_mm
 +POSTHOOK: query: insert overwrite table iow1_mm partition (key2)
 +select key + 3 as k1, key + 3 from intermediate union all select key + 2 as k1, key + 2 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@iow1_mm@key2=100
 +POSTHOOK: Output: default@iow1_mm@key2=101
 +POSTHOOK: Output: default@iow1_mm@key2=102
 +POSTHOOK: Output: default@iow1_mm@key2=103
 +POSTHOOK: Output: default@iow1_mm@key2=105
 +POSTHOOK: Output: default@iow1_mm@key2=106
 +POSTHOOK: Output: default@iow1_mm@key2=12
 +POSTHOOK: Output: default@iow1_mm@key2=13
 +POSTHOOK: Output: default@iow1_mm@key2=2
 +POSTHOOK: Output: default@iow1_mm@key2=3
 +POSTHOOK: Output: default@iow1_mm@key2=99
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=100).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=101).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=102).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=103).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=105).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=106).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=12).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=13).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=2).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=3).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: iow1_mm PARTITION(key2=99).key EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +PREHOOK: query: select * from iow1_mm order by key, key2
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@iow1_mm
 +PREHOOK: Input: default@iow1_mm@key2=0
 +PREHOOK: Input: default@iow1_mm@key2=10
 +PREHOOK: Input: default@iow1_mm@key2=100
 +PREHOOK: Input: default@iow1_mm@key2=101
 +PREHOOK: Input: default@iow1_mm@key2=102
 +PREHOOK: Input: default@iow1_mm@key2=103
 +PREHOOK: Input: default@iow1_mm@key2=105
 +PREHOOK: Input: default@iow1_mm@key2=106
 +PREHOOK: Input: default@iow1_mm@key2=12
 +PREHOOK: Input: default@iow1_mm@key2=13
 +PREHOOK: Input: default@iow1_mm@key2=2
 +PREHOOK: Input: default@iow1_mm@key2=3
 +PREHOOK: Input: default@iow1_mm@key2=97
 +PREHOOK: Input: default@iow1_mm@key2=98
 +PREHOOK: Input: default@iow1_mm@key2=99
 +#### A masked pattern was here ####
 +POSTHOOK: query: select * from iow1_mm order by key, key2
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@iow1_mm
 +POSTHOOK: Input: default@iow1_mm@key2=0
 +POSTHOOK: Input: default@iow1_mm@key2=10
 +POSTHOOK: Input: default@iow1_mm@key2=100
 +POSTHOOK: Input: default@iow1_mm@key2=101
 +POSTHOOK: Input: default@iow1_mm@key2=102
 +POSTHOOK: Input: default@iow1_mm@key2=103
 +POSTHOOK: Input: default@iow1_mm@key2=105
 +POSTHOOK: Input: default@iow1_mm@key2=106
 +POSTHOOK: Input: default@iow1_mm@key2=12
 +POSTHOOK: Input: default@iow1_mm@key2=13
 +POSTHOOK: Input: default@iow1_mm@key2=2
 +POSTHOOK: Input: default@iow1_mm@key2=3
 +POSTHOOK: Input: default@iow1_mm@key2=97
 +POSTHOOK: Input: default@iow1_mm@key2=98
 +POSTHOOK: Input: default@iow1_mm@key2=99
 +#### A masked pattern was here ####
 +2	2
 +3	0
 +3	3
 +4	0
 +12	12
 +13	10
 +13	13
 +14	10
 +99	99
 +100	97
 +100	100
 +100	100
 +101	97
 +101	98
 +101	101
 +102	98
 +102	102
 +103	103
 +105	105
 +106	106
 +PREHOOK: query: drop table iow1_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@iow1_mm
 +PREHOOK: Output: default@iow1_mm
 +POSTHOOK: query: drop table iow1_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@iow1_mm
 +POSTHOOK: Output: default@iow1_mm
 +PREHOOK: query: drop table load0_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table load0_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table load0_mm (key string, value string) stored as textfile tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@load0_mm
 +POSTHOOK: query: create table load0_mm (key string, value string) stored as textfile tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@load0_mm
 +PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table load0_mm
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@load0_mm
 +POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table load0_mm
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@load0_mm
 +PREHOOK: query: select count(1) from load0_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@load0_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select count(1) from load0_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@load0_mm
 +#### A masked pattern was here ####
 +500
 +PREHOOK: query: load data local inpath '../../data/files/kv2.txt' into table load0_mm
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@load0_mm
 +POSTHOOK: query: load data local inpath '../../data/files/kv2.txt' into table load0_mm
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@load0_mm
 +PREHOOK: query: select count(1) from load0_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@load0_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select count(1) from load0_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@load0_mm
 +#### A masked pattern was here ####
 +1000
 +PREHOOK: query: load data local inpath '../../data/files/kv2.txt' overwrite into table load0_mm
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@load0_mm
 +POSTHOOK: query: load data local inpath '../../data/files/kv2.txt' overwrite into table load0_mm
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@load0_mm
 +PREHOOK: query: select count(1) from load0_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@load0_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select count(1) from load0_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@load0_mm
 +#### A masked pattern was here ####
 +500
 +PREHOOK: query: drop table load0_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@load0_mm
 +PREHOOK: Output: default@load0_mm
 +POSTHOOK: query: drop table load0_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@load0_mm
 +POSTHOOK: Output: default@load0_mm
 +PREHOOK: query: drop table intermediate2
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table intermediate2
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table intermediate2 (key string, value string) stored as textfile
 +#### A masked pattern was here ####
 +PREHOOK: type: CREATETABLE
 +#### A masked pattern was here ####
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: create table intermediate2 (key string, value string) stored as textfile
 +#### A masked pattern was here ####
 +POSTHOOK: type: CREATETABLE
 +#### A masked pattern was here ####
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@intermediate2
 +PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +PREHOOK: query: load data local inpath '../../data/files/kv2.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv2.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +PREHOOK: query: load data local inpath '../../data/files/kv3.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv3.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +PREHOOK: query: drop table load1_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table load1_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table load1_mm (key string, value string) stored as textfile tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@load1_mm
 +POSTHOOK: query: create table load1_mm (key string, value string) stored as textfile tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@load1_mm
 +#### A masked pattern was here ####
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@load1_mm
 +#### A masked pattern was here ####
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@load1_mm
 +#### A masked pattern was here ####
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@load1_mm
 +#### A masked pattern was here ####
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@load1_mm
 +PREHOOK: query: select count(1) from load1_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@load1_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select count(1) from load1_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@load1_mm
 +#### A masked pattern was here ####
 +1000
 +PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +PREHOOK: query: load data local inpath '../../data/files/kv2.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv2.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +PREHOOK: query: load data local inpath '../../data/files/kv3.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv3.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +#### A masked pattern was here ####
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@load1_mm
 +#### A masked pattern was here ####
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@load1_mm
 +PREHOOK: query: select count(1) from load1_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@load1_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select count(1) from load1_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@load1_mm
 +#### A masked pattern was here ####
 +1050
 +PREHOOK: query: load data local inpath '../../data/files/kv2.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv2.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +#### A masked pattern was here ####
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@load1_mm
 +#### A masked pattern was here ####
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@load1_mm
 +PREHOOK: query: select count(1) from load1_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@load1_mm
 +#### A masked pattern was here ####
 +POSTHOOK: query: select count(1) from load1_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@load1_mm
 +#### A masked pattern was here ####
 +500
 +PREHOOK: query: drop table load1_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@load1_mm
 +PREHOOK: Output: default@load1_mm
 +POSTHOOK: query: drop table load1_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@load1_mm
 +POSTHOOK: Output: default@load1_mm
 +PREHOOK: query: drop table load2_mm
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table load2_mm
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table load2_mm (key string, value string)
 +  partitioned by (k int, l int) stored as textfile tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@load2_mm
 +POSTHOOK: query: create table load2_mm (key string, value string)
 +  partitioned by (k int, l int) stored as textfile tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@load2_mm
 +PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +PREHOOK: query: load data local inpath '../../data/files/kv2.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv2.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +PREHOOK: query: load data local inpath '../../data/files/kv3.txt' into table intermediate2
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: load data local inpath '../../data/files/kv3.txt' into table intermediate2
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@intermediate2
 +#### A masked pattern was here ####
 +PREHOOK: type: LOAD
 +#### A masked pattern was here ####
 +PREHOOK: Output: default@load2_mm
 +#### A masked pattern was here ####
 +POSTHOOK: type: LOAD
 +#### A masked pattern was here ####
 +POSTHOOK: Output: default@load2_mm
 +POSTHOOK: Output: default@load2_mm@k=5/l=5
 +PREHOOK: query: select count(1) from load2_mm
 +PREHOOK: type: QUERY
 +PREHOOK: Input: default@load2_mm
 +PREHOOK: Input: default@load2_mm@k=5/l=5
 +#### A masked pattern was here ####
 +POSTHOOK: query: select count(1) from load2_mm
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@load2_mm
 +POSTHOOK: Input: default@load2_mm@k=5/l=5
 +#### A masked pattern was here ####
 +1025
 +PREHOOK: query: drop table load2_mm
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@load2_mm
 +PREHOOK: Output: default@load2_mm
 +POSTHOOK: query: drop table load2_mm
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@load2_mm
 +POSTHOOK: Output: default@load2_mm
 +PREHOOK: query: drop table intermediate2
 +PREHOOK: type: DROPTABLE
 +PREHOOK: Input: default@intermediate2
 +PREHOOK: Output: default@intermediate2
 +POSTHOOK: query: drop table intermediate2
 +POSTHOOK: type: DROPTABLE
 +POSTHOOK: Input: default@intermediate2
 +POSTHOOK: Output: default@intermediate2
 +PREHOOK: query: drop table intermediate_nonpart
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table intermediate_nonpart
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: drop table intermmediate_part
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table intermmediate_part
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: drop table intermmediate_nonpart
 +PREHOOK: type: DROPTABLE
 +POSTHOOK: query: drop table intermmediate_nonpart
 +POSTHOOK: type: DROPTABLE
 +PREHOOK: query: create table intermediate_nonpart(key int, p int)
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@intermediate_nonpart
 +POSTHOOK: query: create table intermediate_nonpart(key int, p int)
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@intermediate_nonpart
 +PREHOOK: query: insert into intermediate_nonpart select * 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@intermediate_nonpart
 +POSTHOOK: query: insert into intermediate_nonpart select * 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@intermediate_nonpart
 +POSTHOOK: Lineage: intermediate_nonpart.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 +POSTHOOK: Lineage: intermediate_nonpart.p SIMPLE [(intermediate)intermediate.FieldSchema(name:p, type:int, comment:null), ]
 +PREHOOK: query: create table intermmediate_nonpart(key int, p int) tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +PREHOOK: type: CREATETABLE
 +PREHOOK: Output: database:default
 +PREHOOK: Output: default@intermmediate_nonpart
 +POSTHOOK: query: create table intermmediate_nonpart(key int, p int) tblproperties("transactional"="true", "transactional_properties"="insert_only")
 +POSTHOOK: type: CREATETABLE
 +POSTHOOK: Output: database:default
 +POSTHOOK: Output: default@intermmediate_nonpart
 +PREHOOK: query: insert into intermmediate_nonpart select * 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@intermmediate_nonpart
 +POSTHOOK: query: insert into intermmediate_nonpart select * from intermediate
 +POSTHOOK: type: QUERY
 +POSTHOOK: Input: default@intermediate
 +POSTHOOK: Input: default@intermediate@p=455
 +POSTHOOK: Input: default@intermediate@p=456
 +POS

<TRUNCATED>

[18/50] [abbrv] hive git commit: HIVE-15904: select query throwing Null Pointer Exception from org.apache.hadoop.hive.ql.optimizer.DynamicPartitionPruningOptimization.generateSemiJoinOperatorPlan (Jason Dere, reviewed by Gunther Hagleitner)

Posted by se...@apache.org.
HIVE-15904: select query throwing Null Pointer Exception from org.apache.hadoop.hive.ql.optimizer.DynamicPartitionPruningOptimization.generateSemiJoinOperatorPlan (Jason Dere, reviewed by Gunther Hagleitner)


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

Branch: refs/heads/hive-14535
Commit: 0debf9f2916b2ed115e1cdb392a595ae7cf0c761
Parents: 56f6c9d
Author: Jason Dere <jd...@hortonworks.com>
Authored: Sun Feb 19 15:58:25 2017 -0800
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Sun Feb 19 15:58:25 2017 -0800

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../DynamicPartitionPruningOptimization.java    |  36 ++-
 .../dynamic_semijoin_reduction_2.q              |  41 +++
 .../llap/dynamic_semijoin_reduction_2.q.out     | 301 +++++++++++++++++++
 4 files changed, 365 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0debf9f2/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 7c54275..4a69bcc 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -482,6 +482,7 @@ minillaplocal.query.files=acid_globallimit.q,\
   disable_merge_for_bucketing.q,\
   dynamic_partition_pruning.q,\
   dynamic_semijoin_reduction.q,\
+  dynamic_semijoin_reduction_2.q,\
   dynpart_sort_opt_vectorization.q,\
   dynpart_sort_optimization.q,\
   dynpart_sort_optimization_acid.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/0debf9f2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
index c8691e8..8692c45 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
@@ -398,25 +398,33 @@ public class DynamicPartitionPruningOptimization implements NodeProcessor {
       String internalColName = null;
       ExprNodeDesc exprNodeDesc = key;
       // Find the ExprNodeColumnDesc
-      while (!(exprNodeDesc instanceof ExprNodeColumnDesc)) {
+      while (!(exprNodeDesc instanceof ExprNodeColumnDesc) &&
+              (exprNodeDesc.getChildren() != null)) {
         exprNodeDesc = exprNodeDesc.getChildren().get(0);
       }
-      internalColName = ((ExprNodeColumnDesc) exprNodeDesc).getColumn();
 
-      ExprNodeColumnDesc colExpr = ((ExprNodeColumnDesc)(parentOfRS.
-              getColumnExprMap().get(internalColName)));
-      String colName = ExprNodeDescUtils.extractColName(colExpr);
+      if (exprNodeDesc instanceof ExprNodeColumnDesc) {
+        internalColName = ((ExprNodeColumnDesc) exprNodeDesc).getColumn();
 
-      // Fetch the TableScan Operator.
-      Operator<?> op = parentOfRS.getParentOperators().get(0);
-      while (op != null && !(op instanceof TableScanOperator)) {
-        op = op.getParentOperators().get(0);
-      }
-      assert op != null;
+        ExprNodeColumnDesc colExpr = ((ExprNodeColumnDesc) (parentOfRS.
+                getColumnExprMap().get(internalColName)));
+        String colName = ExprNodeDescUtils.extractColName(colExpr);
+
+        // Fetch the TableScan Operator.
+        Operator<?> op = parentOfRS.getParentOperators().get(0);
+        while (op != null && !(op instanceof TableScanOperator)) {
+          op = op.getParentOperators().get(0);
+        }
+        assert op != null;
 
-      Table table = ((TableScanOperator) op).getConf().getTableMetadata();
-      if (table.isPartitionKey(colName)) {
-        // The column is partition column, skip the optimization.
+        Table table = ((TableScanOperator) op).getConf().getTableMetadata();
+        if (table.isPartitionKey(colName)) {
+          // The column is partition column, skip the optimization.
+          return false;
+        }
+      } else {
+        // No column found!
+        // Bail out
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/0debf9f2/ql/src/test/queries/clientpositive/dynamic_semijoin_reduction_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/dynamic_semijoin_reduction_2.q b/ql/src/test/queries/clientpositive/dynamic_semijoin_reduction_2.q
new file mode 100644
index 0000000..2306395
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/dynamic_semijoin_reduction_2.q
@@ -0,0 +1,41 @@
+set hive.compute.query.using.stats=false;
+set hive.mapred.mode=nonstrict;
+set hive.explain.user=false;
+set hive.optimize.ppd=true;
+set hive.ppd.remove.duplicatefilters=true;
+set hive.tez.dynamic.partition.pruning=true;
+set hive.tez.dynamic.semijoin.reduction=true;
+set hive.optimize.metadataonly=false;
+set hive.optimize.index.filter=true;
+
+CREATE TABLE `table_1`(
+  `bigint_col_7` bigint,
+  `decimal2016_col_26` decimal(20,16),
+  `tinyint_col_3` tinyint,
+  `decimal2612_col_77` decimal(26,12),
+  `timestamp_col_9` timestamp);
+
+CREATE TABLE `table_18`(
+  `tinyint_col_15` tinyint,
+  `decimal2709_col_9` decimal(27,9),
+  `tinyint_col_20` tinyint,
+  `smallint_col_19` smallint,
+  `decimal1911_col_16` decimal(19,11),
+  `timestamp_col_18` timestamp);
+
+-- HIVE-15904
+EXPLAIN
+SELECT
+COUNT(*)
+FROM table_1 t1
+INNER JOIN table_18 t2 ON (((t2.tinyint_col_15) = (t1.bigint_col_7)) AND
+((t2.decimal2709_col_9) = (t1.decimal2016_col_26))) AND
+((t2.tinyint_col_20) = (t1.tinyint_col_3))
+WHERE (t2.smallint_col_19) IN (SELECT
+COALESCE(-92, -994) AS int_col
+FROM table_1 tt1
+INNER JOIN table_18 tt2 ON (tt2.decimal1911_col_16) = (tt1.decimal2612_col_77)
+WHERE (t1.timestamp_col_9) = (tt2.timestamp_col_18));
+
+drop table table_1;
+drop table table_18;

http://git-wip-us.apache.org/repos/asf/hive/blob/0debf9f2/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out
new file mode 100644
index 0000000..d3e0e39
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_2.q.out
@@ -0,0 +1,301 @@
+PREHOOK: query: CREATE TABLE `table_1`(
+  `bigint_col_7` bigint,
+  `decimal2016_col_26` decimal(20,16),
+  `tinyint_col_3` tinyint,
+  `decimal2612_col_77` decimal(26,12),
+  `timestamp_col_9` timestamp)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table_1
+POSTHOOK: query: CREATE TABLE `table_1`(
+  `bigint_col_7` bigint,
+  `decimal2016_col_26` decimal(20,16),
+  `tinyint_col_3` tinyint,
+  `decimal2612_col_77` decimal(26,12),
+  `timestamp_col_9` timestamp)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table_1
+PREHOOK: query: CREATE TABLE `table_18`(
+  `tinyint_col_15` tinyint,
+  `decimal2709_col_9` decimal(27,9),
+  `tinyint_col_20` tinyint,
+  `smallint_col_19` smallint,
+  `decimal1911_col_16` decimal(19,11),
+  `timestamp_col_18` timestamp)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table_18
+POSTHOOK: query: CREATE TABLE `table_18`(
+  `tinyint_col_15` tinyint,
+  `decimal2709_col_9` decimal(27,9),
+  `tinyint_col_20` tinyint,
+  `smallint_col_19` smallint,
+  `decimal1911_col_16` decimal(19,11),
+  `timestamp_col_18` timestamp)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table_18
+PREHOOK: query: EXPLAIN
+SELECT
+COUNT(*)
+FROM table_1 t1
+INNER JOIN table_18 t2 ON (((t2.tinyint_col_15) = (t1.bigint_col_7)) AND
+((t2.decimal2709_col_9) = (t1.decimal2016_col_26))) AND
+((t2.tinyint_col_20) = (t1.tinyint_col_3))
+WHERE (t2.smallint_col_19) IN (SELECT
+COALESCE(-92, -994) AS int_col
+FROM table_1 tt1
+INNER JOIN table_18 tt2 ON (tt2.decimal1911_col_16) = (tt1.decimal2612_col_77)
+WHERE (t1.timestamp_col_9) = (tt2.timestamp_col_18))
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT
+COUNT(*)
+FROM table_1 t1
+INNER JOIN table_18 t2 ON (((t2.tinyint_col_15) = (t1.bigint_col_7)) AND
+((t2.decimal2709_col_9) = (t1.decimal2016_col_26))) AND
+((t2.tinyint_col_20) = (t1.tinyint_col_3))
+WHERE (t2.smallint_col_19) IN (SELECT
+COALESCE(-92, -994) AS int_col
+FROM table_1 tt1
+INNER JOIN table_18 tt2 ON (tt2.decimal1911_col_16) = (tt1.decimal2612_col_77)
+WHERE (t1.timestamp_col_9) = (tt2.timestamp_col_18))
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 10 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 8 <- Map 10 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: t1
+                  filterExpr: (bigint_col_7 is not null and decimal2016_col_26 is not null and tinyint_col_3 is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Filter Operator
+                    predicate: (bigint_col_7 is not null and decimal2016_col_26 is not null and tinyint_col_3 is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: bigint_col_7 (type: bigint), decimal2016_col_26 (type: decimal(20,16)), tinyint_col_3 (type: tinyint), timestamp_col_9 (type: timestamp)
+                      outputColumnNames: _col0, _col1, _col2, _col3
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: decimal(34,16)), _col2 (type: tinyint), _col0 (type: bigint)
+                        sort order: +++
+                        Map-reduce partition columns: _col1 (type: decimal(34,16)), _col2 (type: tinyint), _col0 (type: bigint)
+                        Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                        value expressions: _col3 (type: timestamp)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 10 
+            Map Operator Tree:
+                TableScan
+                  alias: tt2
+                  filterExpr: ((timestamp_col_18 = timestamp_col_18) and decimal1911_col_16 is not null and timestamp_col_18 BETWEEN DynamicValue(RS_23_t1_timestamp_col_18_min) AND DynamicValue(RS_23_t1_timestamp_col_18_max) and in_bloom_filter(timestamp_col_18, DynamicValue(RS_23_t1_timestamp_col_18_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Filter Operator
+                    predicate: ((timestamp_col_18 = timestamp_col_18) and decimal1911_col_16 is not null and timestamp_col_18 BETWEEN DynamicValue(RS_23_t1_timestamp_col_18_min) AND DynamicValue(RS_23_t1_timestamp_col_18_max) and in_bloom_filter(timestamp_col_18, DynamicValue(RS_23_t1_timestamp_col_18_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: decimal1911_col_16 (type: decimal(19,11)), timestamp_col_18 (type: timestamp)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: decimal(26,12))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: decimal(26,12))
+                        Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                        value expressions: _col1 (type: timestamp)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: t2
+                  filterExpr: (tinyint_col_15 is not null and decimal2709_col_9 is not null and tinyint_col_20 is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Filter Operator
+                    predicate: (tinyint_col_15 is not null and decimal2709_col_9 is not null and tinyint_col_20 is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: tinyint_col_15 (type: tinyint), decimal2709_col_9 (type: decimal(27,9)), tinyint_col_20 (type: tinyint), smallint_col_19 (type: smallint)
+                      outputColumnNames: _col0, _col1, _col2, _col3
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: decimal(34,16)), _col2 (type: tinyint), UDFToLong(_col0) (type: bigint)
+                        sort order: +++
+                        Map-reduce partition columns: _col1 (type: decimal(34,16)), _col2 (type: tinyint), UDFToLong(_col0) (type: bigint)
+                        Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                        value expressions: _col3 (type: smallint)
+            Execution mode: llap
+            LLAP IO: no inputs
+        Map 7 
+            Map Operator Tree:
+                TableScan
+                  alias: tt1
+                  filterExpr: decimal2612_col_77 is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Filter Operator
+                    predicate: decimal2612_col_77 is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    Select Operator
+                      expressions: decimal2612_col_77 (type: decimal(26,12))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: decimal(26,12))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: decimal(26,12))
+                        Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Execution mode: llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col1 (type: decimal(34,16)), _col2 (type: tinyint), _col0 (type: bigint)
+                  1 _col1 (type: decimal(34,16)), _col2 (type: tinyint), UDFToLong(_col0) (type: bigint)
+                outputColumnNames: _col3, _col7
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col3 (type: timestamp), UDFToInteger(_col7) (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col3 (type: timestamp), UDFToInteger(_col7) (type: int)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Select Operator
+                  expressions: _col3 (type: timestamp)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Group By Operator
+                    aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=1)
+                    mode: hash
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col3 (type: timestamp), UDFToInteger(_col7) (type: int)
+                  1 _col1 (type: timestamp), -92 (type: int)
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 4 
+            Execution mode: 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: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 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
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=1)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary)
+        Reducer 8 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: decimal(26,12))
+                  1 _col0 (type: decimal(26,12))
+                outputColumnNames: _col2
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Group By Operator
+                  keys: _col2 (type: timestamp)
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: timestamp)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: timestamp)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+        Reducer 9 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: timestamp)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: timestamp)
+                  outputColumnNames: _col1
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: timestamp), -92 (type: int)
+                    sort order: ++
+                    Map-reduce partition columns: _col1 (type: timestamp), -92 (type: int)
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: drop table table_1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@table_1
+PREHOOK: Output: default@table_1
+POSTHOOK: query: drop table table_1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@table_1
+POSTHOOK: Output: default@table_1
+PREHOOK: query: drop table table_18
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@table_18
+PREHOOK: Output: default@table_18
+POSTHOOK: query: drop table table_18
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@table_18
+POSTHOOK: Output: default@table_18


[39/50] [abbrv] hive git commit: HIVE-15955: make explain formatted to include opId and etc (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/vector_outer_join6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_outer_join6.q.out b/ql/src/test/results/clientpositive/vector_outer_join6.q.out
index a910ed2..bf5d503 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join6.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join6.q.out
@@ -130,7 +130,7 @@ POSTHOOK: query: explain vectorization detail formatted
 select tj1rnum, tj2rnum, tjoin3.rnum as rnumt3 from
    (select tjoin1.rnum tj1rnum, tjoin2.rnum tj2rnum, tjoin2.c1 tj2c1 from tjoin1 left outer join tjoin2 on tjoin1.c1 = tjoin2.c1 ) tj left outer join tjoin3 on tj2c1 = tjoin3.c1
 POSTHOOK: type: QUERY
-{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-7":{"ROOT STAGE":"TRUE"},"Stage-5":{"DEPENDENT STAGES":"Stage-7"},"Stage-0":{"DEPENDENT STAGES":"Stage-5"}},"STAGE PLANS":{"Stage-7":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_0:$hdt$_1:tjoin2":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_1:tjoin3":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_0:$hdt$_1:tjoin2":{"TableScan":{"alias:":"tjoin2","Statistics:":"Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: int)","1":"_col1 (type: int)"}}}}}}},"$hdt$_1:tjoin3":{"TableScan":{"alias:":"tjoin3","Statistics:":"Num rows:
  2 Data size: 188 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col2 (type: int)","1":"_col1 (type: int)"}}}}}}}}}},"Stage-5":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"tjoin1","Statistics:":"Num rows: 3 Data size: 32 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1]"},"Statistics:":"Num rows: 3 Data size: 32 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Out
 er Join0 to 1"}],"keys:":{"0":"_col1 (type: int)","1":"_col1 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col2","_col3"],"Statistics:":"Num rows: 4 Data size: 409 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"_col0 (type: int), _col2 (type: int), _col3 (type: int)","outputColumnNames:":["_col0","_col1","_col2"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"Statistics:":"Num rows: 4 Data size: 409 Basic stats: COMPLETE Column stats: NONE","children":{"
 Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col2 (type: int)","1":"_col1 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col1","_col3"],"Statistics:":"Num rows: 4 Data size: 449 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"_col0 (type: int), _col1 (type: int), _col3 (type: int)","outputColumnNames:":["_col0","_col1","_col2"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"Statistics:":"Num rows: 4 Data size: 449 Basi
 c stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","File Sink Vectorization:":{"className:":"VectorFileSinkOperator","native:":"false"},"Statistics:":"Num rows: 4 Data size: 449 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:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"3","includeColumns:":"[0, 1]","dataColumns:":["rnum:int","c1:int","c2:int"],"partitionColumnCount:":"0","scratchColumnTypeNames:":["b
 igint","bigint"]}},"Local Work:":{"Map Reduce Local Work":{}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-7":{"ROOT STAGE":"TRUE"},"Stage-5":{"DEPENDENT STAGES":"Stage-7"},"Stage-0":{"DEPENDENT STAGES":"Stage-5"}},"STAGE PLANS":{"Stage-7":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_0:$hdt$_1:tjoin2":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_1:tjoin3":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_0:$hdt$_1:tjoin2":{"TableScan":{"alias:":"tjoin2","Statistics:":"Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: int)","1":"_col1 (type: int)"},"OperatorId:":"HASHTABLESINK_21"}}}}
 }},"$hdt$_1:tjoin3":{"TableScan":{"alias:":"tjoin3","Statistics:":"Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_8","children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_9","children":{"HashTable Sink Operator":{"keys:":{"0":"_col2 (type: int)","1":"_col1 (type: int)"},"OperatorId:":"HASHTABLESINK_19"}}}}}}}}},"Stage-5":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"tjoin1","Statistics:":"Num rows: 3 Data size: 32 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projected
 OutputColumns:":"[0, 1]"},"Statistics:":"Num rows: 3 Data size: 32 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_23","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: int)","1":"_col1 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col2","_col3"],"Statistics:":"Num rows: 4 Data size: 409 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_24","children":{"Select Operator":{"expressions:":"_col0 (type: int), _col2 (type: int), _col3 (type: int)","outputColumnNames:":["_col0","_col1",
 "_col2"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"Statistics:":"Num rows: 4 Data size: 409 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_25","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col2 (type: int)","1":"_col1 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col1","_col3"],"Statistics:":"Num rows: 4 Data size: 449 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_26","children":{"Select Operator":{"expressio
 ns:":"_col0 (type: int), _col1 (type: int), _col3 (type: int)","outputColumnNames:":["_col0","_col1","_col2"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"Statistics:":"Num rows: 4 Data size: 449 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_27","children":{"File Output Operator":{"compressed:":"false","File Sink Vectorization:":{"className:":"VectorFileSinkOperator","native:":"false"},"Statistics:":"Num rows: 4 Data size: 449 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"},"OperatorId:":"FS_28"}}}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","
 inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"3","includeColumns:":"[0, 1]","dataColumns:":["rnum:int","c1:int","c2:int"],"partitionColumnCount:":"0","scratchColumnTypeNames:":["bigint","bigint"]}},"Local Work:":{"Map Reduce Local Work":{}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_29"}}}}}}
 PREHOOK: query: select tj1rnum, tj2rnum, tjoin3.rnum as rnumt3 from
    (select tjoin1.rnum tj1rnum, tjoin2.rnum tj2rnum, tjoin2.c1 tj2c1 from tjoin1 left outer join tjoin2 on tjoin1.c1 = tjoin2.c1 ) tj left outer join tjoin3 on tj2c1 = tjoin3.c1
 PREHOOK: type: QUERY
@@ -157,7 +157,7 @@ POSTHOOK: query: explain vectorization detail formatted
 select tj1rnum, tj2rnum as rnumt3 from
    (select tjoin1.rnum tj1rnum, tjoin2.rnum tj2rnum, tjoin2.c1 tj2c1 from tjoin1 left outer join tjoin2 on tjoin1.c1 = tjoin2.c1 ) tj left outer join tjoin3 on tj2c1 = tjoin3.c1
 POSTHOOK: type: QUERY
-{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-7":{"ROOT STAGE":"TRUE"},"Stage-5":{"DEPENDENT STAGES":"Stage-7"},"Stage-0":{"DEPENDENT STAGES":"Stage-5"}},"STAGE PLANS":{"Stage-7":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_0:$hdt$_1:tjoin2":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_1:tjoin3":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_0:$hdt$_1:tjoin2":{"TableScan":{"alias:":"tjoin2","Statistics:":"Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: int)","1":"_col1 (type: int)"}}}}}}},"$hdt$_1:tjoin3":{"TableScan":{"alias:":"tjoin3","Statistics:":"Num rows:
  2 Data size: 188 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"c1 (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col2 (type: int)","1":"_col0 (type: int)"}}}}}}}}}},"Stage-5":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"tjoin1","Statistics:":"Num rows: 3 Data size: 32 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1]"},"Statistics:":"Num rows: 3 Data size: 32 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{
 "0":"_col1 (type: int)","1":"_col1 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col2","_col3"],"Statistics:":"Num rows: 4 Data size: 409 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"_col0 (type: int), _col2 (type: int), _col3 (type: int)","outputColumnNames:":["_col0","_col1","_col2"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"Statistics:":"Num rows: 4 Data size: 409 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condi
 tion map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col2 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 4 Data size: 449 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","File Sink Vectorization:":{"className:":"VectorFileSinkOperator","native:":"false"},"Statistics:":"Num rows: 4 Data size: 449 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:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"3","includeColumns:":"[0, 1]","dataColumns:":["rnum:int","c1:int","c2:int"],"partitionColumnCount:":"0","scratchColumnTypeNames:":["bigint","bigint"]}},"Local Work:":{"Map Reduce Local Work":{}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-7":{"ROOT STAGE":"TRUE"},"Stage-5":{"DEPENDENT STAGES":"Stage-7"},"Stage-0":{"DEPENDENT STAGES":"Stage-5"}},"STAGE PLANS":{"Stage-7":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_0:$hdt$_1:tjoin2":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_1:tjoin3":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_0:$hdt$_1:tjoin2":{"TableScan":{"alias:":"tjoin2","Statistics:":"Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: int)","1":"_col1 (type: int)"},"OperatorId:":"HASHTABLESINK_21"}}}}
 }},"$hdt$_1:tjoin3":{"TableScan":{"alias:":"tjoin3","Statistics:":"Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_8","children":{"Select Operator":{"expressions:":"c1 (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_9","children":{"HashTable Sink Operator":{"keys:":{"0":"_col2 (type: int)","1":"_col0 (type: int)"},"OperatorId:":"HASHTABLESINK_19"}}}}}}}}},"Stage-5":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"tjoin1","Statistics:":"Num rows: 3 Data size: 32 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"rnum (type: int), c1 (type: int)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1]"},
 "Statistics:":"Num rows: 3 Data size: 32 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_23","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: int)","1":"_col1 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col2","_col3"],"Statistics:":"Num rows: 4 Data size: 409 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_24","children":{"Select Operator":{"expressions:":"_col0 (type: int), _col2 (type: int), _col3 (type: int)","outputColumnNames:":["_col0","_col1","_col2"],"Select Vectoriza
 tion:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1, 2]"},"Statistics:":"Num rows: 4 Data size: 409 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_25","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col2 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 4 Data size: 449 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_26","children":{"File Output Operator":{"compressed:":"false","File Sink Vector
 ization:":{"className:":"VectorFileSinkOperator","native:":"false"},"Statistics:":"Num rows: 4 Data size: 449 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"},"OperatorId:":"FS_27"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"3","includeColumns:":"[0, 1]","dataColumns:":["rnum:int","c1:int","c2:int"],"partitionColumnCount:":"0","scratchColumnTypeNames:":["bigint","bigint"]}},"Local Work:":{"Map Reduce Local Work":{}}}},"Stage-0":{"Fetch Operator":{
 "limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_28"}}}}}}
 PREHOOK: query: select tj1rnum, tj2rnum as rnumt3 from
    (select tjoin1.rnum tj1rnum, tjoin2.rnum tj2rnum, tjoin2.c1 tj2c1 from tjoin1 left outer join tjoin2 on tjoin1.c1 = tjoin2.c1 ) tj left outer join tjoin3 on tj2c1 = tjoin3.c1
 PREHOOK: type: QUERY


[30/50] [abbrv] hive git commit: HIVE-15971: LLAP: logs urls should use daemon container id instead of fake container id

Posted by se...@apache.org.
HIVE-15971: LLAP: logs urls should use daemon container id instead of fake container id


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

Branch: refs/heads/hive-14535
Commit: d5bb76cf2da3934d1de6b3087ac4bfafa2b2cb6f
Parents: de532b1
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Feb 21 14:25:47 2017 -0800
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Feb 21 14:25:47 2017 -0800

----------------------------------------------------------------------
 .../llap/registry/impl/LlapRegistryService.java | 13 +--
 .../llap/tezplugins/LlapTaskCommunicator.java   | 91 +++++++++++---------
 2 files changed, 57 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d5bb76cf/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapRegistryService.java
----------------------------------------------------------------------
diff --git a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapRegistryService.java b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapRegistryService.java
index 5a94db9..610c0a5 100644
--- a/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapRegistryService.java
+++ b/llap-client/src/java/org/apache/hadoop/hive/llap/registry/impl/LlapRegistryService.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.llap.registry.ServiceInstanceSet;
 import org.apache.hadoop.hive.llap.registry.ServiceInstanceStateChangeListener;
 import org.apache.hadoop.hive.llap.registry.ServiceRegistry;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.slf4j.Logger;
@@ -57,17 +58,17 @@ public class LlapRegistryService extends AbstractService {
     String hosts = HiveConf.getTrimmedVar(conf, HiveConf.ConfVars.LLAP_DAEMON_SERVICE_HOSTS);
     Preconditions.checkNotNull(hosts, ConfVars.LLAP_DAEMON_SERVICE_HOSTS.toString() + " must be defined");
     LlapRegistryService registry;
-    // TODO: this is not going to work with multiple users.
     if (hosts.startsWith("@")) {
       // Caching instances only in case of the YARN registry. Each host based list will get it's own copy.
-      String name = hosts.substring(1);
-      if (yarnRegistries.containsKey(name) && yarnRegistries.get(name).isInState(STATE.STARTED)) {
-        registry = yarnRegistries.get(name);
-      } else {
+      String appName = hosts.substring(1);
+      String userName = HiveConf.getVar(conf, ConfVars.LLAP_ZK_REGISTRY_USER, RegistryUtils.currentUser());
+      String key = appName + "-" + userName;
+      registry = yarnRegistries.get(key);
+      if (registry == null || !registry.isInState(STATE.STARTED)) {
         registry = new LlapRegistryService(false);
         registry.init(conf);
         registry.start();
-        yarnRegistries.put(name, registry);
+        yarnRegistries.put(key, registry);
       }
     } else {
       registry = new LlapRegistryService(false);

http://git-wip-us.apache.org/repos/asf/hive/blob/d5bb76cf/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
----------------------------------------------------------------------
diff --git a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
index 3aae7a4..e593b33 100644
--- a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
+++ b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskCommunicator.java
@@ -14,12 +14,12 @@
 
 package org.apache.hadoop.hive.llap.tezplugins;
 
+import org.apache.hadoop.hive.llap.registry.ServiceInstance;
 import org.apache.hadoop.io.Writable;
 
 import org.apache.hadoop.hive.llap.protocol.LlapTaskUmbilicalProtocol.TezAttemptArray;
 
 import java.io.IOException;
-import java.net.URI;
 import java.nio.ByteBuffer;
 import java.util.HashSet;
 import java.util.Map;
@@ -75,7 +75,7 @@ import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.security.JobTokenSecretManager;
@@ -119,7 +119,8 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
   private final Token<LlapTokenIdentifier> token;
   private final String user;
   private String amHost;
-  private URI timelineServerUri;
+  private String timelineServerUri;
+  private int nmPort;
 
   // These two structures track the list of known nodes, and the list of nodes which are sending in keep-alive heartbeats.
   // Primarily for debugging purposes a.t.m, since there's some unexplained TASK_TIMEOUTS which are currently being observed.
@@ -149,7 +150,6 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
     Preconditions.checkState((token != null) == UserGroupInformation.isSecurityEnabled());
 
     // Not closing this at the moment at shutdown, since this could be a shared instance.
-    // TODO: this is unused.
     serviceRegistry = LlapRegistryService.getClient(conf);
 
     umbilical = new LlapTaskUmbilicalProtocolImpl(getUmbilical());
@@ -191,18 +191,10 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
         + "fileCleanupDelay=" + deleteDelayOnDagComplete
         + ", numCommunicatorThreads=" + numThreads);
     this.communicator.init(conf);
-    if (YarnConfiguration.useHttps(conf)) {
-      timelineServerUri = URI
-        .create(JOINER.join("https://", conf.get(
-          YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS),
-          RESOURCE_URI_STR));
-    } else {
-      timelineServerUri = URI.create(JOINER.join("http://", conf.get(
-        YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS),
-        RESOURCE_URI_STR));
-    }
+    String scheme = WebAppUtils.getHttpSchemePrefix(conf);
+    String ahsUrl = WebAppUtils.getAHSWebAppURLWithoutScheme(conf);
+    this.timelineServerUri = WebAppUtils.getURLWithScheme(scheme, ahsUrl);
+    this.nmPort = Integer.valueOf(WebAppUtils.getNMWebAppURLWithoutScheme(conf).split(":")[1]);
   }
 
   @Override
@@ -540,37 +532,54 @@ public class LlapTaskCommunicator extends TezTaskCommunicatorImpl {
 
   @Override
   public String getInProgressLogsUrl(TezTaskAttemptID attemptID, NodeId containerNodeId) {
-    String url = "";
-    if (timelineServerUri != null && containerNodeId != null) {
-      LlapNodeId llapNodeId = LlapNodeId.getInstance(containerNodeId.getHost(), containerNodeId.getPort());
-      BiMap<ContainerId, TezTaskAttemptID> biMap = entityTracker.getContainerAttemptMapForNode(llapNodeId);
-      ContainerId containerId = biMap.inverse().get(attemptID);
-      if (containerId != null) {
-        String dagId = attemptID.getTaskID().getVertexID().getDAGId().toString();
-        String filename = currentHiveQueryId + "-" + dagId + ".log";
-        // YARN-6011 provides a webservice to get the logs
-        url = PATH_JOINER.join(timelineServerUri.toString(), "containers", containerId.toString(), "logs",
-          filename);
-      }
-    }
-    return url;
+    return constructLogUrl(attemptID, containerNodeId, false);
   }
 
   @Override
   public String getCompletedLogsUrl(TezTaskAttemptID attemptID, NodeId containerNodeId) {
-    String url = "";
-    if (timelineServerUri != null && containerNodeId != null) {
-      LlapNodeId llapNodeId = LlapNodeId.getInstance(containerNodeId.getHost(), containerNodeId.getPort());
-      BiMap<ContainerId, TezTaskAttemptID> biMap = entityTracker.getContainerAttemptMapForNode(llapNodeId);
-      ContainerId containerId = biMap.inverse().get(attemptID);
-      if (containerId != null) {
-        String dagId = attemptID.getTaskID().getVertexID().getDAGId().toString();
-        String filename = currentHiveQueryId + "-" + dagId + ".log.done";
-        // YARN-6011 provides a webservice to get the logs
-        url = PATH_JOINER.join(timelineServerUri.toString(), "containers", containerId.toString(), "logs",
-          filename);
+    return constructLogUrl(attemptID, containerNodeId, true);
+  }
+
+  private String constructLogUrl(final TezTaskAttemptID attemptID, final NodeId containerNodeId, final boolean isDone) {
+    if (timelineServerUri == null || containerNodeId == null) {
+      return null;
+    }
+    Set<ServiceInstance> instanceSet;
+    try {
+      instanceSet = serviceRegistry.getInstances().getByHost(containerNodeId.getHost());
+    } catch (IOException e) {
+      // Not failing the job due to a failure constructing the log url
+      LOG.warn(
+        "Unable to find instance for yarnNodeId={} to construct the log url. Exception message={}",
+        containerNodeId, e.getMessage());
+      return null;
+    }
+    if (instanceSet != null) {
+      ServiceInstance matchedInstance = null;
+      for (ServiceInstance instance : instanceSet) {
+        if (instance.getRpcPort() == containerNodeId.getPort()) {
+          matchedInstance = instance;
+          break;
+        }
+      }
+      if (matchedInstance != null) {
+        String containerIdString = matchedInstance.getProperties()
+          .get(HiveConf.ConfVars.LLAP_DAEMON_CONTAINER_ID.varname);
+        if (containerIdString != null) {
+          return constructLlapLogUrl(attemptID, containerIdString, isDone, containerNodeId.getHost());
+        }
       }
     }
+    return null;
+  }
+
+  private String constructLlapLogUrl(final TezTaskAttemptID attemptID, final String containerIdString,
+    final boolean isDone, final String nmHost) {
+    String dagId = attemptID.getTaskID().getVertexID().getDAGId().toString();
+    String filename = JOINER.join(currentHiveQueryId, "-", dagId, ".log", (isDone ? ".done" : ""),
+      "?nm.id=", nmHost, ":", nmPort);
+    String url = PATH_JOINER.join(timelineServerUri, "ws", "v1", "applicationhistory", "containers",
+      containerIdString, "logs", filename);
     return url;
   }
 


[11/50] [abbrv] hive git commit: HIVE-15954 : LLAP: some Tez INFO logs are too noisy (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-15954 : LLAP: some Tez INFO logs are too noisy (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/bba18181
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/bba18181
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/bba18181

Branch: refs/heads/hive-14535
Commit: bba18181ad107fbea17c2d2bea9fd88873a4a943
Parents: 6f6a558
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri Feb 17 11:49:38 2017 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri Feb 17 11:50:14 2017 -0800

----------------------------------------------------------------------
 .../src/main/resources/llap-daemon-log4j2.properties  | 14 +++++++++++++-
 1 file changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bba18181/llap-server/src/main/resources/llap-daemon-log4j2.properties
----------------------------------------------------------------------
diff --git a/llap-server/src/main/resources/llap-daemon-log4j2.properties b/llap-server/src/main/resources/llap-daemon-log4j2.properties
index a9bfa34..31a34be 100644
--- a/llap-server/src/main/resources/llap-daemon-log4j2.properties
+++ b/llap-server/src/main/resources/llap-daemon-log4j2.properties
@@ -100,7 +100,19 @@ appender.query-routing.routes.route-mdc.file-mdc.app.layout.type = PatternLayout
 appender.query-routing.routes.route-mdc.file-mdc.app.layout.pattern = %d{ISO8601} %5p [%t (%X{fragmentId})] %c{2}: %m%n
 
 # list of all loggers
-loggers = PerfLogger, EncodedReader, NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX, HistoryLogger, LlapIoImpl, LlapIoOrc, LlapIoCache, LlapIoLocking
+loggers = PerfLogger, EncodedReader, NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX, HistoryLogger, LlapIoImpl, LlapIoOrc, LlapIoCache, LlapIoLocking, TezSM, TezSS, TezM, TezHC
+
+
+# shut up the Tez logs that log debug-level stuff on INFO
+
+logger.TezSM.name = org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager
+logger.TezSM.level = WARN
+logger.TezSS.name = org.apache.tez.runtime.library.common.shuffle.orderedgrouped.ShuffleScheduler
+logger.TezSS.level = WARN
+logger.TezM.name = org.apache.tez.runtime.library.common.sort.impl.TezMerger
+logger.TezM.level = WARN
+logger.TezHC.name = org.apache.tez.http.HttpConnection
+logger.TezHC.level = WARN
 
 logger.PerfLogger.name = org.apache.hadoop.hive.ql.log.PerfLogger
 logger.PerfLogger.level = DEBUG


[04/50] [abbrv] hive git commit: HIVE-15902: Select query involving date throwing Hive 2 Internal error: unsupported conversion from type: date (Jason Dere, reviewed by Matt McCline)

Posted by se...@apache.org.
HIVE-15902: Select query involving date throwing Hive 2 Internal error: unsupported conversion from type: date (Jason Dere, reviewed by Matt McCline)


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

Branch: refs/heads/hive-14535
Commit: f3790ce2604621ac2512053b50eaca1ad053f178
Parents: e49a074
Author: Jason Dere <jd...@hortonworks.com>
Authored: Thu Feb 16 13:52:29 2017 -0800
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Thu Feb 16 13:52:29 2017 -0800

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../FilterColumnBetweenDynamicValue.txt         |   9 +-
 .../ql/exec/vector/VectorizationContext.java    |   2 +-
 .../vectorized_dynamic_semijoin_reduction2.q    |  50 ++
 ...vectorized_dynamic_semijoin_reduction2.q.out | 772 +++++++++++++++++++
 .../apache/hadoop/hive/tools/GenVectorCode.java |  18 +
 6 files changed, 847 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 2c53047..e8db920 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -610,6 +610,7 @@ minillaplocal.query.files=acid_globallimit.q,\
   vectorization_short_regress.q,\
   vectorized_dynamic_partition_pruning.q,\
   vectorized_dynamic_semijoin_reduction.q,\
+  vectorized_dynamic_semijoin_reduction2.q,\
   vectorized_ptf.q,\
   windowing.q,\
   windowing_gby.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
index 97ab7aa..1aee9b3 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
@@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.Filter<TypeName>ColumnBetween;
 import org.apache.hadoop.hive.ql.plan.DynamicValue;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -79,12 +80,12 @@ public class <ClassName> extends Filter<TypeName>ColumnBetween {
       if (lVal == null || rVal == null) {
         isLeftOrRightNull = true;
       } else {
-        <VectorType> min = PrimitiveObjectInspectorUtils.<GetPrimitiveMethod>(
-            lVal, leftDynamicValue.getObjectInspector())<GetValueMethod>;
+        <VectorType> min = <ConversionMethod>(PrimitiveObjectInspectorUtils.<GetPrimitiveMethod>(
+            lVal, leftDynamicValue.getObjectInspector())<GetValueMethod>);
         setLeftValue(min);
 
-        <VectorType> max = PrimitiveObjectInspectorUtils.<GetPrimitiveMethod>(
-            rVal, rightDynamicValue.getObjectInspector())<GetValueMethod>;
+        <VectorType> max = <ConversionMethod>(PrimitiveObjectInspectorUtils.<GetPrimitiveMethod>(
+            rVal, rightDynamicValue.getObjectInspector())<GetValueMethod>);
         setRightValue(max);
       }
       initialized = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 4802489..8164684 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -2240,7 +2240,7 @@ public class VectorizationContext {
       cl = FilterDecimalColumnNotBetween.class;
     } else if (isDateFamily(colType) && !notKeywordPresent) {
       cl =  (hasDynamicValues ?
-          FilterLongColumnBetweenDynamicValue.class :
+          FilterDateColumnBetweenDynamicValue.class :
           FilterLongColumnBetween.class);
     } else if (isDateFamily(colType) && notKeywordPresent) {
       cl = FilterLongColumnNotBetween.class;

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q b/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
new file mode 100644
index 0000000..446407d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
@@ -0,0 +1,50 @@
+set hive.compute.query.using.stats=false;
+set hive.mapred.mode=nonstrict;
+set hive.explain.user=false;
+set hive.optimize.ppd=true;
+set hive.ppd.remove.duplicatefilters=true;
+set hive.tez.dynamic.partition.pruning=true;
+set hive.tez.dynamic.semijoin.reduction=true;
+set hive.optimize.metadataonly=false;
+set hive.optimize.index.filter=true;
+
+set hive.vectorized.adaptor.usage.mode=none;
+set hive.vectorized.execution.enabled=true;
+
+-- Create Tables
+create table dsrv2_big stored as orc as
+  select
+  cast(L_PARTKEY as bigint) as partkey_bigint,
+  cast(L_PARTKEY as decimal(10,1)) as partkey_decimal,
+  cast(L_PARTKEY as double) as partkey_double,
+  cast(l_shipdate as date) as shipdate_date,
+  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts
+  from lineitem;
+create table dsrv2_small stored as orc as select * from dsrv2_big limit 20;
+analyze table dsrv2_big compute statistics;
+analyze table dsrv2_small compute statistics;
+analyze table dsrv2_big compute statistics for columns;
+analyze table dsrv2_small compute statistics for columns;
+
+-- single key (bigint)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint);
+
+-- single key (decimal)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal);
+
+-- single key (double)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double);
+
+-- single key (date)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date);
+
+-- single key (timestamp)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts);
+
+drop table dsrv2_big;
+drop table dsrv2_small;

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
new file mode 100644
index 0000000..27d8152
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
@@ -0,0 +1,772 @@
+PREHOOK: query: create table dsrv2_big stored as orc as
+  select
+  cast(L_PARTKEY as bigint) as partkey_bigint,
+  cast(L_PARTKEY as decimal(10,1)) as partkey_decimal,
+  cast(L_PARTKEY as double) as partkey_double,
+  cast(l_shipdate as date) as shipdate_date,
+  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts
+  from lineitem
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@lineitem
+PREHOOK: Output: database:default
+PREHOOK: Output: default@dsrv2_big
+POSTHOOK: query: create table dsrv2_big stored as orc as
+  select
+  cast(L_PARTKEY as bigint) as partkey_bigint,
+  cast(L_PARTKEY as decimal(10,1)) as partkey_decimal,
+  cast(L_PARTKEY as double) as partkey_double,
+  cast(l_shipdate as date) as shipdate_date,
+  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts
+  from lineitem
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@lineitem
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@dsrv2_big
+POSTHOOK: Lineage: dsrv2_big.partkey_bigint EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_partkey, type:int, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.partkey_decimal EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_partkey, type:int, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.partkey_double EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_partkey, type:int, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.shipdate_date EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_shipdate, type:string, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.shipdate_ts EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_shipdate, type:string, comment:null), ]
+PREHOOK: query: create table dsrv2_small stored as orc as select * from dsrv2_big limit 20
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Output: database:default
+PREHOOK: Output: default@dsrv2_small
+POSTHOOK: query: create table dsrv2_small stored as orc as select * from dsrv2_big limit 20
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@dsrv2_small
+POSTHOOK: Lineage: dsrv2_small.partkey_bigint SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:partkey_bigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.partkey_decimal SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:partkey_decimal, type:decimal(10,1), comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.partkey_double SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:partkey_double, type:double, comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.shipdate_date SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:shipdate_date, type:date, comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.shipdate_ts SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:shipdate_ts, type:timestamp, comment:null), ]
+PREHOOK: query: analyze table dsrv2_big compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Output: default@dsrv2_big
+POSTHOOK: query: analyze table dsrv2_big compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Output: default@dsrv2_big
+PREHOOK: query: analyze table dsrv2_small compute statistics
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_small
+PREHOOK: Output: default@dsrv2_small
+POSTHOOK: query: analyze table dsrv2_small compute statistics
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_small
+POSTHOOK: Output: default@dsrv2_small
+PREHOOK: query: analyze table dsrv2_big compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table dsrv2_big compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+#### A masked pattern was here ####
+PREHOOK: query: analyze table dsrv2_small compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table dsrv2_small compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (partkey_bigint is not null and partkey_bigint BETWEEN DynamicValue(RS_7_b_partkey_bigint_min) AND DynamicValue(RS_7_b_partkey_bigint_max) and in_bloom_filter(partkey_bigint, DynamicValue(RS_7_b_partkey_bigint_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (partkey_bigint is not null and partkey_bigint BETWEEN DynamicValue(RS_7_b_partkey_bigint_min) AND DynamicValue(RS_7_b_partkey_bigint_max) and in_bloom_filter(partkey_bigint, DynamicValue(RS_7_b_partkey_bigint_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_bigint (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: bigint)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: bigint)
+                        Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: partkey_bigint is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: partkey_bigint is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_bigint (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: bigint)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: bigint)
+                        Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: bigint)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=18)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: bigint)
+                  1 _col0 (type: bigint)
+                Statistics: Num rows: 20 Data size: 160 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)
+        Reducer 3 
+            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
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=18)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_bigint = b.partkey_bigint)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+20
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (partkey_decimal is not null and partkey_decimal BETWEEN DynamicValue(RS_7_b_partkey_decimal_min) AND DynamicValue(RS_7_b_partkey_decimal_max) and in_bloom_filter(partkey_decimal, DynamicValue(RS_7_b_partkey_decimal_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (partkey_decimal is not null and partkey_decimal BETWEEN DynamicValue(RS_7_b_partkey_decimal_min) AND DynamicValue(RS_7_b_partkey_decimal_max) and in_bloom_filter(partkey_decimal, DynamicValue(RS_7_b_partkey_decimal_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_decimal (type: decimal(10,1))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: decimal(10,1))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: decimal(10,1))
+                        Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: partkey_decimal is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: partkey_decimal is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_decimal (type: decimal(10,1))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: decimal(10,1))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: decimal(10,1))
+                        Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: decimal(10,1))
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=16)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: decimal(10,1)), _col1 (type: decimal(10,1)), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: decimal(10,1))
+                  1 _col0 (type: decimal(10,1))
+                Statistics: Num rows: 20 Data size: 160 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)
+        Reducer 3 
+            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
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=16)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: decimal(10,1)), _col1 (type: decimal(10,1)), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_decimal = b.partkey_decimal)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+20
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (partkey_double is not null and partkey_double BETWEEN DynamicValue(RS_7_b_partkey_double_min) AND DynamicValue(RS_7_b_partkey_double_max) and in_bloom_filter(partkey_double, DynamicValue(RS_7_b_partkey_double_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (partkey_double is not null and partkey_double BETWEEN DynamicValue(RS_7_b_partkey_double_min) AND DynamicValue(RS_7_b_partkey_double_max) and in_bloom_filter(partkey_double, DynamicValue(RS_7_b_partkey_double_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_double (type: double)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: double)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: double)
+                        Statistics: Num rows: 100 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: partkey_double is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: partkey_double is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: partkey_double (type: double)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: double)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: double)
+                        Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: double)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=30)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: double)
+                  1 _col0 (type: double)
+                Statistics: Num rows: 20 Data size: 160 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)
+        Reducer 3 
+            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
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=30)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.partkey_double = b.partkey_double)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+20
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (shipdate_date is not null and shipdate_date BETWEEN DynamicValue(RS_7_b_shipdate_date_min) AND DynamicValue(RS_7_b_shipdate_date_max) and in_bloom_filter(shipdate_date, DynamicValue(RS_7_b_shipdate_date_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (shipdate_date is not null and shipdate_date BETWEEN DynamicValue(RS_7_b_shipdate_date_min) AND DynamicValue(RS_7_b_shipdate_date_max) and in_bloom_filter(shipdate_date, DynamicValue(RS_7_b_shipdate_date_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_date (type: date)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: date)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: date)
+                        Statistics: Num rows: 100 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: shipdate_date is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: shipdate_date is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_date (type: date)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: date)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: date)
+                        Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: date)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 1120 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=20)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: date)
+                  1 _col0 (type: date)
+                Statistics: Num rows: 2000 Data size: 16000 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)
+        Reducer 3 
+            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
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=20)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.shipdate_date)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+23
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (shipdate_ts is not null and shipdate_ts BETWEEN DynamicValue(RS_7_b_shipdate_ts_min) AND DynamicValue(RS_7_b_shipdate_ts_max) and in_bloom_filter(shipdate_ts, DynamicValue(RS_7_b_shipdate_ts_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (shipdate_ts is not null and shipdate_ts BETWEEN DynamicValue(RS_7_b_shipdate_ts_min) AND DynamicValue(RS_7_b_shipdate_ts_max) and in_bloom_filter(shipdate_ts, DynamicValue(RS_7_b_shipdate_ts_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_ts (type: timestamp)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: timestamp)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: timestamp)
+                        Statistics: Num rows: 100 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: shipdate_ts is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: shipdate_ts is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_ts (type: timestamp)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: timestamp)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: timestamp)
+                        Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: timestamp)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 800 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=20)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: timestamp)
+                  1 _col0 (type: timestamp)
+                Statistics: Num rows: 2000 Data size: 16000 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)
+        Reducer 3 
+            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
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=20)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: timestamp), _col1 (type: timestamp), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+23
+PREHOOK: query: drop table dsrv2_big
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Output: default@dsrv2_big
+POSTHOOK: query: drop table dsrv2_big
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Output: default@dsrv2_big
+PREHOOK: query: drop table dsrv2_small
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@dsrv2_small
+PREHOOK: Output: default@dsrv2_small
+POSTHOOK: query: drop table dsrv2_small
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@dsrv2_small
+POSTHOOK: Output: default@dsrv2_small

http://git-wip-us.apache.org/repos/asf/hive/blob/f3790ce2/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
----------------------------------------------------------------------
diff --git a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
index 22b8752..55cfb7b 100644
--- a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
+++ b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
@@ -796,6 +796,7 @@ public class GenVectorCode extends Task {
       {"FilterColumnBetweenDynamicValue", "string", ""},
       {"FilterColumnBetweenDynamicValue", "char", ""},
       {"FilterColumnBetweenDynamicValue", "varchar", ""},
+      {"FilterColumnBetweenDynamicValue", "date", ""},
       {"FilterColumnBetweenDynamicValue", "timestamp", ""},
 
       {"ColumnCompareColumn", "Equal", "long", "double", "=="},
@@ -1402,42 +1403,58 @@ public class GenVectorCode extends Task {
     String vectorType;
     String getPrimitiveMethod;
     String getValueMethod;
+    String conversionMethod;
 
     if (operandType.equals("long")) {
       defaultValue = "0";
       vectorType = "long";
       getPrimitiveMethod = "getLong";
       getValueMethod = "";
+      conversionMethod = "";
     } else if (operandType.equals("double")) {
       defaultValue = "0";
       vectorType = "double";
       getPrimitiveMethod = "getDouble";
       getValueMethod = "";
+      conversionMethod = "";
     } else if (operandType.equals("decimal")) {
       defaultValue = "null";
       vectorType = "HiveDecimal";
       getPrimitiveMethod = "getHiveDecimal";
       getValueMethod = "";
+      conversionMethod = "";
     } else if (operandType.equals("string")) {
       defaultValue = "null";
       vectorType = "byte[]";
       getPrimitiveMethod = "getString";
       getValueMethod = ".getBytes()";
+      conversionMethod = "";
     } else if (operandType.equals("char")) {
       defaultValue = "null";
       vectorType = "byte[]";
       getPrimitiveMethod = "getHiveChar";
       getValueMethod = ".getStrippedValue().getBytes()";  // Does vectorization use stripped char values?
+      conversionMethod = "";
     } else if (operandType.equals("varchar")) {
       defaultValue = "null";
       vectorType = "byte[]";
       getPrimitiveMethod = "getHiveVarchar";
       getValueMethod = ".getValue().getBytes()";
+      conversionMethod = "";
+    } else if (operandType.equals("date")) {
+      defaultValue = "0";
+      vectorType = "long";
+      getPrimitiveMethod = "getDate";
+      getValueMethod = "";
+      conversionMethod = "DateWritable.dateToDays";
+      // Special case - Date requires its own specific BetweenDynamicValue class, but derives from FilterLongColumnBetween
+      typeName = "Long";
     } else if (operandType.equals("timestamp")) {
       defaultValue = "null";
       vectorType = "Timestamp";
       getPrimitiveMethod = "getTimestamp";
       getValueMethod = "";
+      conversionMethod = "";
     } else {
       throw new IllegalArgumentException("Type " + operandType + " not supported");
     }
@@ -1451,6 +1468,7 @@ public class GenVectorCode extends Task {
     templateString = templateString.replaceAll("<VectorType>", vectorType);
     templateString = templateString.replaceAll("<GetPrimitiveMethod>", getPrimitiveMethod);
     templateString = templateString.replaceAll("<GetValueMethod>", getValueMethod);
+    templateString = templateString.replaceAll("<ConversionMethod>", conversionMethod);
 
     writeFile(templateFile.lastModified(), expressionOutputDirectory, expressionClassesDirectory,
         className, templateString);


[06/50] [abbrv] hive git commit: HIVE-15710: HS2 Stopped when running in background (Rui reviewed by Ferdinand, Xuefu and Mohit)

Posted by se...@apache.org.
HIVE-15710: HS2 Stopped when running in background (Rui reviewed by Ferdinand, Xuefu and Mohit)


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

Branch: refs/heads/hive-14535
Commit: 90688bde9b1172fb2a042905dba645fbeda92510
Parents: ef61a9b
Author: Rui Li <li...@apache.org>
Authored: Fri Feb 17 11:08:40 2017 +0800
Committer: Rui Li <sh...@cn.ibm.com>
Committed: Fri Feb 17 11:08:40 2017 +0800

----------------------------------------------------------------------
 bin/beeline    | 5 -----
 bin/ext/cli.sh | 8 --------
 bin/hive       | 7 +++++++
 3 files changed, 7 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/90688bde/bin/beeline
----------------------------------------------------------------------
diff --git a/bin/beeline b/bin/beeline
index 7b974d4..d247c39 100644
--- a/bin/beeline
+++ b/bin/beeline
@@ -18,9 +18,4 @@
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 
-# If process is backgrounded, don't change terminal settings
-if [[ ( ! $(ps -o stat= -p $$) =~ "+" ) && ! ( -p /dev/stdin ) ]]; then
-  export HADOOP_CLIENT_OPTS="$HADOOP_CLIENT_OPTS -Djline.terminal=jline.UnsupportedTerminal"
-fi
-
 . "$bin"/hive --service beeline "$@"

http://git-wip-us.apache.org/repos/asf/hive/blob/90688bde/bin/ext/cli.sh
----------------------------------------------------------------------
diff --git a/bin/ext/cli.sh b/bin/ext/cli.sh
index 4933c2e..87329f3 100644
--- a/bin/ext/cli.sh
+++ b/bin/ext/cli.sh
@@ -22,13 +22,6 @@ if [ -z "$USE_DEPRECATED_CLI" ] || [ "$USE_DEPRECATED_CLI" != "false" ]; then
   USE_DEPRECATED_CLI="true"
 fi
 
-updateBeelineOpts() {
-  # If process is backgrounded, don't change terminal settings
-  if [[ ( ! $(ps -o stat= -p $$) =~ *+ ) && ! ( -p /dev/stdin ) ]]; then
-    export HADOOP_CLIENT_OPTS="$HADOOP_CLIENT_OPTS -Djline.terminal=jline.UnsupportedTerminal"
-  fi
-}
-
 updateCli() {
   if [ "$USE_DEPRECATED_CLI" == "true" ]; then
     CLASS=org.apache.hadoop.hive.cli.CliDriver
@@ -37,7 +30,6 @@ updateCli() {
     export HADOOP_CLIENT_OPTS="$HADOOP_CLIENT_OPTS -Dlog4j.configurationFile=beeline-log4j2.properties"
     CLASS=org.apache.hive.beeline.cli.HiveCli
     JAR=hive-beeline-*.jar
-    updateBeelineOpts
   fi
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/90688bde/bin/hive
----------------------------------------------------------------------
diff --git a/bin/hive b/bin/hive
index 50fbddd..e1ee206 100755
--- a/bin/hive
+++ b/bin/hive
@@ -344,6 +344,13 @@ else
   export HADOOP_CLIENT_OPTS="$HADOOP_CLIENT_OPTS -Dlog4j.configurationFile=hive-log4j2.properties -Djava.util.logging.config.file=$bin/../conf/parquet-logging.properties "
 fi
 
+if [[ "$SERVICE" =~ ^(hiveserver2|beeline|cli)$ ]] ; then
+  # If process is backgrounded, don't change terminal settings
+  if [[ ( ! $(ps -o stat= -p $$) =~ "+" ) && ! ( -p /dev/stdin ) ]]; then
+    export HADOOP_CLIENT_OPTS="$HADOOP_CLIENT_OPTS -Djline.terminal=jline.UnsupportedTerminal"
+  fi
+fi
+
 if [ "$TORUN" = "" ] ; then
   echo "Service $SERVICE not found"
   echo "Available Services: $SERVICE_LIST"


[46/50] [abbrv] hive git commit: HIVE-15830. Allow additional ACLs for tez jobs. (Siddharth Seth, reviewed by Daniel Dai)

Posted by se...@apache.org.
HIVE-15830. Allow additional ACLs for tez jobs. (Siddharth Seth, reviewed by Daniel Dai)


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

Branch: refs/heads/hive-14535
Commit: 657236ee0cce12d6e02bf0c2c406fc5dbf7b3791
Parents: 539d3c6
Author: Siddharth Seth <ss...@apache.org>
Authored: Thu Feb 23 14:25:07 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Thu Feb 23 14:25:07 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   2 +
 .../common/util/ACLConfigurationParser.java     | 167 +++++++++++++++++++
 .../common/util/TestACLConfigurationParser.java |  99 +++++++++++
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   3 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |  23 +++
 .../hive/ql/exec/tez/TezSessionState.java       |  27 +++
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |  34 +++-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |   2 +-
 .../hadoop/hive/ql/exec/tez/TestTezTask.java    |   6 +
 9 files changed, 354 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/657236ee/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 46be3fb..f0c129b 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1462,6 +1462,8 @@ public class HiveConf extends Configuration {
     HIVETEZLOGLEVEL("hive.tez.log.level", "INFO",
         "The log level to use for tasks executing as part of the DAG.\n" +
         "Used only if hive.tez.java.opts is used to configure Java options."),
+    HIVETEZHS2USERACCESS("hive.tez.hs2.user.access", true,
+        "Whether to grant access to the hs2/hive user for queries"),
     HIVEQUERYNAME ("hive.query.name", null,
         "This named is used by Tez to set the dag name. This name in turn will appear on \n" +
         "the Tez UI representing the work that was done."),

http://git-wip-us.apache.org/repos/asf/hive/blob/657236ee/common/src/java/org/apache/hive/common/util/ACLConfigurationParser.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/common/util/ACLConfigurationParser.java b/common/src/java/org/apache/hive/common/util/ACLConfigurationParser.java
new file mode 100644
index 0000000..cb1eaf9
--- /dev/null
+++ b/common/src/java/org/apache/hive/common/util/ACLConfigurationParser.java
@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.common.util;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.collect.Sets;
+
+/**
+ * Parser for extracting ACL information from Configs
+ */
+@Private
+public class ACLConfigurationParser {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ACLConfigurationParser.class);
+
+  private static final String WILDCARD_ACL_VALUE = "*";
+  private static final Pattern splitPattern = Pattern.compile("\\s+");
+
+  private final Set<String> allowedUsers;
+  private final Set<String> allowedGroups;
+
+  public ACLConfigurationParser(Configuration conf, String confPropertyName) {
+    allowedUsers = Sets.newLinkedHashSet();
+    allowedGroups = Sets.newLinkedHashSet();
+    parse(conf, confPropertyName);
+  }
+
+
+  private boolean isWildCard(String aclStr) {
+    return aclStr.trim().equals(WILDCARD_ACL_VALUE);
+  }
+
+  private void parse(Configuration conf, String configProperty) {
+    String aclsStr = conf.get(configProperty);
+    if (aclsStr == null || aclsStr.isEmpty()) {
+      return;
+    }
+    if (isWildCard(aclsStr)) {
+      allowedUsers.add(WILDCARD_ACL_VALUE);
+      return;
+    }
+
+    final String[] splits = splitPattern.split(aclsStr);
+    int counter = -1;
+    String userListStr = null;
+    String groupListStr = null;
+    for (String s : splits) {
+      if (s.isEmpty()) {
+        if (userListStr != null) {
+          continue;
+        }
+      }
+      ++counter;
+      if (counter == 0) {
+        userListStr = s;
+      } else if (counter == 1) {
+        groupListStr = s;
+      } else {
+        LOG.warn("Invalid configuration specified for " + configProperty
+            + ", ignoring configured ACLs, value=" + aclsStr);
+        return;
+      }
+    }
+
+    if (userListStr == null) {
+      return;
+    }
+    if (userListStr.length() >= 1) {
+      allowedUsers.addAll(
+          org.apache.hadoop.util.StringUtils.getTrimmedStringCollection(userListStr));
+    }
+    if (groupListStr != null && groupListStr.length() >= 1) {
+      allowedGroups.addAll(
+          org.apache.hadoop.util.StringUtils.getTrimmedStringCollection(groupListStr));
+    }
+  }
+
+  public Set<String> getAllowedUsers() {
+    return Collections.unmodifiableSet(allowedUsers);
+  }
+
+  public Set<String> getAllowedGroups() {
+    return Collections.unmodifiableSet(allowedGroups);
+  }
+
+  public void addAllowedUser(String user) {
+    if (StringUtils.isBlank(user)) {
+      return;
+    }
+    if (allowedUsers.contains(WILDCARD_ACL_VALUE)) {
+      return;
+    }
+    if (user.equals(WILDCARD_ACL_VALUE)) {
+      allowedUsers.clear();
+      allowedGroups.clear();
+    }
+    allowedUsers.add(user);
+  }
+
+  public void addAllowedGroup(String group) {
+    allowedGroups.add(group);
+  }
+
+  public String toAclString() {
+    return toString();
+  }
+
+  @Override
+  public String toString() {
+    if (getAllowedUsers().contains(WILDCARD_ACL_VALUE)) {
+      return WILDCARD_ACL_VALUE;
+    } else {
+      if (allowedUsers.size() == 0 && allowedGroups.size() == 0) {
+        return " ";
+      }
+      String userString = constructCsv(allowedUsers);
+      String groupString = "";
+      if (allowedGroups.size() > 0) {
+        groupString = " " + constructCsv(allowedGroups);
+      }
+      return userString + groupString;
+    }
+  }
+
+  private String constructCsv(Set<String> inSet) {
+    StringBuilder sb = new StringBuilder();
+    if (inSet != null) {
+      boolean isFirst = true;
+      for (String s : inSet) {
+        if (!isFirst) {
+          sb.append(",");
+        } else {
+          isFirst = false;
+        }
+        sb.append(s);
+      }
+    }
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/657236ee/common/src/test/org/apache/hive/common/util/TestACLConfigurationParser.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hive/common/util/TestACLConfigurationParser.java b/common/src/test/org/apache/hive/common/util/TestACLConfigurationParser.java
new file mode 100644
index 0000000..f23573f
--- /dev/null
+++ b/common/src/test/org/apache/hive/common/util/TestACLConfigurationParser.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.common.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+public class TestACLConfigurationParser {
+
+
+  @Test (timeout = 10_000L)
+  public void test() {
+
+    ACLConfigurationParser aclConf;
+    Configuration conf = new Configuration();
+    conf.set("ACL_ALL_ACCESS", "*");
+    aclConf = new ACLConfigurationParser(conf, "ACL_ALL_ACCESS");
+    assertEquals(1, aclConf.getAllowedUsers().size());
+    assertTrue(aclConf.getAllowedUsers().contains("*"));
+    assertEquals(0, aclConf.getAllowedGroups().size());
+    assertEquals("*", aclConf.toAclString());
+
+    conf.set("ACL_INVALID1", "u1, u2, u3");
+    aclConf = new ACLConfigurationParser(conf, "ACL_INVALID1");
+    assertEquals(0, aclConf.getAllowedUsers().size());
+    assertEquals(0, aclConf.getAllowedGroups().size());
+    assertEquals(" ", aclConf.toAclString());
+
+    conf.set("ACL_NONE", " ");
+    aclConf = new ACLConfigurationParser(conf, "ACL_NONE");
+    assertEquals(0, aclConf.getAllowedUsers().size());
+    assertEquals(0, aclConf.getAllowedGroups().size());
+    assertEquals(" ", aclConf.toAclString());
+
+    conf.set("ACL_VALID1", "user1,user2");
+    aclConf = new ACLConfigurationParser(conf, "ACL_VALID1");
+    assertEquals(2, aclConf.getAllowedUsers().size());
+    assertTrue(aclConf.getAllowedUsers().contains("user1"));
+    assertTrue(aclConf.getAllowedUsers().contains("user2"));
+    assertEquals(0, aclConf.getAllowedGroups().size());
+    assertEquals("user1,user2", aclConf.toAclString());
+
+    conf.set("ACL_VALID2", "user1,user2 group1,group2");
+    aclConf = new ACLConfigurationParser(conf, "ACL_VALID2");
+    assertEquals(2, aclConf.getAllowedUsers().size());
+    assertTrue(aclConf.getAllowedUsers().contains("user1"));
+    assertTrue(aclConf.getAllowedUsers().contains("user2"));
+    assertEquals(2, aclConf.getAllowedGroups().size());
+    assertTrue(aclConf.getAllowedGroups().contains("group1"));
+    assertTrue(aclConf.getAllowedGroups().contains("group2"));
+    assertEquals("user1,user2 group1,group2", aclConf.toAclString());
+
+
+    conf.set("ACL_VALID3", "user1 group1");
+    aclConf = new ACLConfigurationParser(conf, "ACL_VALID3");
+    assertEquals(1, aclConf.getAllowedUsers().size());
+    assertTrue(aclConf.getAllowedUsers().contains("user1"));
+    assertEquals(1, aclConf.getAllowedGroups().size());
+    assertTrue(aclConf.getAllowedGroups().contains("group1"));
+    assertEquals("user1 group1", aclConf.toAclString());
+
+    aclConf.addAllowedUser("user2");
+    assertEquals(2, aclConf.getAllowedUsers().size());
+    assertTrue(aclConf.getAllowedUsers().contains("user1"));
+    assertTrue(aclConf.getAllowedUsers().contains("user2"));
+    assertEquals("user1,user2 group1", aclConf.toAclString());
+
+    aclConf.addAllowedGroup("group2");
+    assertEquals(2, aclConf.getAllowedGroups().size());
+    assertTrue(aclConf.getAllowedGroups().contains("group1"));
+    assertTrue(aclConf.getAllowedGroups().contains("group2"));
+    assertEquals("user1,user2 group1,group2", aclConf.toAclString());
+
+    aclConf.addAllowedUser("*");
+    assertEquals(1, aclConf.getAllowedUsers().size());
+    assertTrue(aclConf.getAllowedUsers().contains("*"));
+    assertTrue(aclConf.getAllowedGroups().isEmpty());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/657236ee/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 592b1f1..cdf24d4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -1747,7 +1747,8 @@ public class Driver implements CommandProcessor {
       resStream = null;
 
       SessionState ss = SessionState.get();
-      hookContext = new HookContext(plan, queryState, ctx.getPathToCS(), ss.getUserName(),
+
+      hookContext = new HookContext(plan, queryState, ctx.getPathToCS(), ss.getUserFromAuthenticator(),
           ss.getUserIpAddress(), InetAddress.getLocalHost().getHostAddress(), operationId,
           ss.getSessionId(), Thread.currentThread().getName(), ss.isHiveServerQuery(), perfLogger);
       hookContext.setHookType(HookContext.HookType.PRE_EXEC_HOOK);

http://git-wip-us.apache.org/repos/asf/hive/blob/657236ee/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 e81cbce..3484493 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
@@ -143,6 +143,7 @@ import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Shell;
+import org.apache.hive.common.util.ACLConfigurationParser;
 import org.apache.hive.common.util.ReflectionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -3794,4 +3795,26 @@ public final class Utilities {
     String suffix = "KMGTPE".charAt(exp-1) + "";
     return String.format("%.2f%sB", bytes / Math.pow(unit, exp), suffix);
   }
+
+
+  public static String getAclStringWithHiveModification(Configuration tezConf,
+                                                        String propertyName,
+                                                        boolean addHs2User,
+                                                        String user,
+                                                        String hs2User) throws
+      IOException {
+
+    // Start with initial ACLs
+    ACLConfigurationParser aclConf =
+        new ACLConfigurationParser(tezConf, propertyName);
+
+    // Always give access to the user
+    aclConf.addAllowedUser(user);
+
+    // Give access to the process user if the config is set.
+    if (addHs2User && hs2User != null) {
+      aclConf.addAllowedUser(hs2User);
+    }
+    return aclConf.toAclString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/657236ee/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
index 62f65c2..ed1ba9c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezSessionState.java
@@ -308,6 +308,8 @@ public class TezSessionState {
       tezConfig.setInt(TezConfiguration.TEZ_AM_SESSION_MIN_HELD_CONTAINERS, n);
     }
 
+    setupSessionAcls(tezConfig, conf);
+
     final TezClient session = TezClient.newBuilder("HIVE-" + sessionId, tezConfig)
         .setIsSession(true).setLocalResources(commonLocalResources)
         .setCredentials(llapCredentials).setServicePluginDescriptor(servicePluginsDescriptor)
@@ -433,6 +435,31 @@ public class TezSessionState {
     }
   }
 
+  private void setupSessionAcls(Configuration tezConf, HiveConf hiveConf) throws
+      IOException {
+
+    String user = SessionState.getUserFromAuthenticator();
+    UserGroupInformation loginUserUgi = UserGroupInformation.getLoginUser();
+    String loginUser =
+        loginUserUgi == null ? null : loginUserUgi.getShortUserName();
+    boolean addHs2User =
+        HiveConf.getBoolVar(hiveConf, ConfVars.HIVETEZHS2USERACCESS);
+
+    String viewStr = Utilities.getAclStringWithHiveModification(tezConf,
+            TezConfiguration.TEZ_AM_VIEW_ACLS, addHs2User, user, loginUser);
+    String modifyStr = Utilities.getAclStringWithHiveModification(tezConf,
+            TezConfiguration.TEZ_AM_MODIFY_ACLS, addHs2User, user, loginUser);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          "Setting Tez Session access for sessionId={} with viewAclString={}, modifyStr={}",
+          SessionState.get().getSessionId(), viewStr, modifyStr);
+    }
+
+    tezConf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewStr);
+    tezConf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyStr);
+  }
+
   public void refreshLocalResourcesFromConf(HiveConf conf)
     throws IOException, LoginException, IllegalArgumentException, URISyntaxException, TezException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/657236ee/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index 58f0b33..740e41b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -33,6 +33,7 @@ import java.util.Set;
 import javax.annotation.Nullable;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.metrics.common.Metrics;
@@ -57,6 +58,7 @@ import org.apache.hadoop.hive.ql.plan.UnionWork;
 import org.apache.hadoop.hive.ql.plan.api.StageType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.LocalResource;
@@ -71,6 +73,7 @@ import org.apache.tez.dag.api.DAG;
 import org.apache.tez.dag.api.Edge;
 import org.apache.tez.dag.api.GroupInputEdge;
 import org.apache.tez.dag.api.SessionNotRunning;
+import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.dag.api.VertexGroup;
@@ -348,7 +351,7 @@ public class TezTask extends Task<TezWork> {
     dag.setDAGInfo(dagInfo);
 
     dag.setCredentials(conf.getCredentials());
-    setAccessControlsForCurrentUser(dag);
+    setAccessControlsForCurrentUser(dag, queryPlan.getQueryId(), conf);
 
     for (BaseWork w: ws) {
 
@@ -431,14 +434,31 @@ public class TezTask extends Task<TezWork> {
     return dag;
   }
 
-  public static void setAccessControlsForCurrentUser(DAG dag) {
-    // get current user
-    String currentUser = SessionState.getUserFromAuthenticator();
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Setting Tez DAG access for " + currentUser);
+  private static void setAccessControlsForCurrentUser(DAG dag, String queryId,
+                                                     Configuration conf) throws
+      IOException {
+    String user = SessionState.getUserFromAuthenticator();
+    UserGroupInformation loginUserUgi = UserGroupInformation.getLoginUser();
+    String loginUser =
+        loginUserUgi == null ? null : loginUserUgi.getShortUserName();
+    boolean addHs2User =
+        HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVETEZHS2USERACCESS);
+
+    // Temporarily re-using the TEZ AM View ACLs property for individual dag access control.
+    // Hive may want to setup it's own parameters if it wants to control per dag access.
+    // Setting the tez-property per dag should work for now.
+
+    String viewStr = Utilities.getAclStringWithHiveModification(conf,
+            TezConfiguration.TEZ_AM_VIEW_ACLS, addHs2User, user, loginUser);
+    String modifyStr = Utilities.getAclStringWithHiveModification(conf,
+            TezConfiguration.TEZ_AM_MODIFY_ACLS, addHs2User, user, loginUser);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Setting Tez DAG access for queryId={} with viewAclString={}, modifyStr={}",
+          queryId, viewStr, modifyStr);
     }
     // set permissions for current user on DAG
-    DAGAccessControls ac = new DAGAccessControls(currentUser, currentUser);
+    DAGAccessControls ac = new DAGAccessControls(viewStr, modifyStr);
     dag.setAccessControls(ac);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/657236ee/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
index 72a1acc..13ccd93 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
@@ -162,7 +162,7 @@ public class ATSHook implements ExecuteWithHookContext {
               String queryId = plan.getQueryId();
               String opId = hookContext.getOperationId();
               long queryStartTime = plan.getQueryStartTime();
-              String user = hookContext.getUgi().getUserName();
+              String user = hookContext.getUgi().getShortUserName();
               String requestuser = hookContext.getUserName();
               if (hookContext.getUserName() == null ){
                 requestuser = hookContext.getUgi().getUserName() ;

http://git-wip-us.apache.org/repos/asf/hive/blob/657236ee/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
index 5c012f3..2b52056 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -36,6 +37,7 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -44,6 +46,7 @@ import org.apache.hadoop.hive.common.metrics.common.Metrics;
 import org.apache.hadoop.hive.common.metrics.common.MetricsConstant;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
@@ -161,6 +164,9 @@ public class TestTezTask {
     task = new TezTask(utils);
     task.setWork(work);
     task.setConsole(mock(LogHelper.class));
+    QueryPlan mockQueryPlan = mock(QueryPlan.class);
+    doReturn(UUID.randomUUID().toString()).when(mockQueryPlan).getQueryId();
+    task.setQueryPlan(mockQueryPlan);
 
     conf = new JobConf();
     appLr = mock(LocalResource.class);


[29/50] [abbrv] hive git commit: HIVE-15959 : LLAP: fix headroom calculation and move it to daemon (Sergey Shelukhin, reviewed by Siddharth Seth)

Posted by se...@apache.org.
HIVE-15959 : LLAP: fix headroom calculation and move it to daemon (Sergey Shelukhin, reviewed by Siddharth Seth)


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

Branch: refs/heads/hive-14535
Commit: de532b1f9bb21daa668dac0f2b4f2429c9b4bd37
Parents: af606ff
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Feb 21 13:56:17 2017 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Feb 21 13:56:17 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   | 12 ++--
 .../hadoop/hive/llap/cli/LlapServiceDriver.java | 64 ++++++++------------
 .../hive/llap/daemon/impl/LlapDaemon.java       | 49 +++++++++++----
 llap-server/src/main/resources/package.py       |  8 +--
 .../hive/llap/daemon/MiniLlapCluster.java       |  2 +-
 5 files changed, 72 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/de532b1f/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 1af59ba..4faaa8b 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -381,7 +381,7 @@ public class HiveConf extends Configuration {
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_RPC_PORT.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname);
-    llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_HEADROOM_MEMORY_PER_INSTANCE_MB.varname);
+    llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_XMX_HEADROOM.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_VCPUS_PER_INSTANCE.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_NUM_FILE_CLEANER_THREADS.varname);
     llapDaemonVarsSetLocal.add(ConfVars.LLAP_FILE_CLEANUP_DELAY_SECONDS.varname);
@@ -3072,11 +3072,11 @@ public class HiveConf extends Configuration {
     LLAP_DAEMON_MEMORY_PER_INSTANCE_MB("hive.llap.daemon.memory.per.instance.mb", 4096,
       "The total amount of memory to use for the executors inside LLAP (in megabytes).",
       "llap.daemon.memory.per.instance.mb"),
-    LLAP_DAEMON_HEADROOM_MEMORY_PER_INSTANCE_MB("hive.llap.daemon.headroom.memory.per.instance.mb", 512,
-      "The total amount of memory deducted from daemon memory required for other LLAP services. The remaining memory" +
-      " will be used by the executors. If the cache is off-heap, Executor memory + Headroom memory = Xmx. If the " +
-        "cache is on-heap, Executor memory + Cache memory + Headroom memory = Xmx. The headroom memory has to be " +
-        "minimum of 5% from the daemon memory."),
+    LLAP_DAEMON_XMX_HEADROOM("hive.llap.daemon.xmx.headroom", "5%",
+      "The total amount of heap memory set aside by LLAP and not used by the executors. Can\n" +
+      "be specified as size (e.g. '512Mb'), or percentage (e.g. '5%'). Note that the latter is\n" +
+      "derived from the total daemon XMX, which can be different from the total executor\n" +
+      "memory if the cache is on-heap; although that's not the default configuration."),
     LLAP_DAEMON_VCPUS_PER_INSTANCE("hive.llap.daemon.vcpus.per.instance", 4,
       "The total number of vcpus to use for the executors inside LLAP.",
       "llap.daemon.vcpus.per.instance"),

http://git-wip-us.apache.org/repos/asf/hive/blob/de532b1f/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 a93d53a..e8517ab 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
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.llap.cli;
 
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStreamWriter;
@@ -48,17 +47,10 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hive.llap.LlapUtil;
 import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration;
 import org.apache.hadoop.hive.llap.daemon.impl.LlapConstants;
-import org.apache.hadoop.hive.llap.daemon.impl.LlapDaemon;
 import org.apache.hadoop.hive.llap.daemon.impl.StaticPermanentFunctionChecker;
 import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos;
 import org.apache.hadoop.hive.llap.tezplugins.LlapTezUtils;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
-import org.apache.slider.client.SliderClient;
-import org.apache.slider.common.params.ActionCreateArgs;
-import org.apache.slider.common.params.ActionDestroyArgs;
-import org.apache.slider.common.params.ActionFreezeArgs;
-import org.apache.slider.common.params.ActionInstallPackageArgs;
-import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -84,7 +76,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.eclipse.jetty.server.ssl.SslSocketConnector;
 import org.joda.time.DateTime;
 import org.json.JSONException;
@@ -244,6 +235,7 @@ public class LlapServiceDriver {
         HiveConf.setVar(conf, ConfVars.LLAP_DAEMON_LOGGER, options.getLogger());
         propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_LOGGER.varname, options.getLogger());
       }
+      boolean isDirect = HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_DIRECT);
 
       if (options.getSize() != -1) {
         if (options.getCache() != -1) {
@@ -263,8 +255,7 @@ public class LlapServiceDriver {
               + " smaller than the container sizing (" + LlapUtil.humanReadableByteCount(options.getSize())
               + ")");
         }
-        if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_DIRECT)
-            && false == HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_MAPPED)) {
+        if (isDirect && !HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_ALLOCATOR_MAPPED)) {
           // direct and not memory mapped
           Preconditions.checkArgument(options.getXmx() + options.getCache() <= options.getSize(),
             "Working memory (Xmx=" + LlapUtil.humanReadableByteCount(options.getXmx()) + ") + cache size ("
@@ -273,19 +264,6 @@ public class LlapServiceDriver {
         }
       }
 
-      // 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 ("
-            + LlapUtil.humanReadableByteCount(options.getSize()) + ") should be greater"
-            + " than minimum allocation(" + LlapUtil.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));
-      }
 
       if (options.getExecutors() != -1) {
         conf.setLong(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname, options.getExecutors());
@@ -319,17 +297,30 @@ public class LlapServiceDriver {
             String.valueOf(xmxMb));
       }
 
-      final long currentHeadRoom = options.getSize() - options.getXmx() - options.getCache();
-      final long minHeadRoom = (long) (options.getXmx() * LlapDaemon.MIN_HEADROOM_PERCENT);
-      final long headRoom = currentHeadRoom < minHeadRoom ? minHeadRoom : currentHeadRoom;
-      final long headRoomMb = headRoom / (1024L * 1024L);
-      conf.setLong(ConfVars.LLAP_DAEMON_HEADROOM_MEMORY_PER_INSTANCE_MB.varname, headRoomMb);
-      propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_HEADROOM_MEMORY_PER_INSTANCE_MB.varname,
-        String.valueOf(headRoomMb));
-
-      LOG.info("Memory settings: container memory: {} executor memory: {} cache memory: {} headroom memory: {}",
-        LlapUtil.humanReadableByteCount(options.getSize()), LlapUtil.humanReadableByteCount(options.getXmx()),
-        LlapUtil.humanReadableByteCount(options.getCache()), LlapUtil.humanReadableByteCount(headRoom));
+      long size = options.getSize();
+      if (size == -1) {
+        long heapSize = xmx;
+        if (!isDirect) {
+          heapSize += cache;
+        }
+        size = Math.min((long)(heapSize * 1.2), heapSize + 1024L*1024*1024);
+        if (isDirect) {
+          size += cache;
+        }
+      }
+      long containerSize = size / (1024 * 1024);
+      final long minAlloc = conf.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1);
+      Preconditions.checkArgument(containerSize >= minAlloc, "Container size ("
+          + LlapUtil.humanReadableByteCount(options.getSize()) + ") should be greater"
+          + " than minimum allocation(" + LlapUtil.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));
+
+      LOG.info("Memory settings: container memory: {} executor memory: {} cache memory: {}",
+        LlapUtil.humanReadableByteCount(options.getSize()),
+        LlapUtil.humanReadableByteCount(options.getXmx()),
+        LlapUtil.humanReadableByteCount(options.getCache()));
 
       if (options.getLlapQueueName() != null && !options.getLlapQueueName().isEmpty()) {
         conf.set(ConfVars.LLAP_DAEMON_QUEUE_NAME.varname, options.getLlapQueueName());
@@ -642,9 +633,6 @@ public class LlapServiceDriver {
     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_HEADROOM_MEMORY_PER_INSTANCE_MB.varname,
-      HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_HEADROOM_MEMORY_PER_INSTANCE_MB));
-
     configs.put(ConfVars.LLAP_DAEMON_VCPUS_PER_INSTANCE.varname,
         HiveConf.getIntVar(conf, ConfVars.LLAP_DAEMON_VCPUS_PER_INSTANCE));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/de532b1f/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
index e737fdd..fc9f530 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/LlapDaemon.java
@@ -83,7 +83,6 @@ import com.google.common.primitives.Ints;
 public class LlapDaemon extends CompositeService implements ContainerRunner, LlapDaemonMXBean {
 
   private static final Logger LOG = LoggerFactory.getLogger(LlapDaemon.class);
-  public static final double MIN_HEADROOM_PERCENT = 0.05;
 
   private final Configuration shuffleHandlerConf;
   private final SecretManager secretManager;
@@ -114,7 +113,7 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
 
   public LlapDaemon(Configuration daemonConf, int numExecutors, long executorMemoryBytes,
     boolean ioEnabled, boolean isDirectCache, long ioMemoryBytes, String[] localDirs, int srvPort,
-    int mngPort, int shufflePort, int webPort, String appName, final long headRoomBytes) {
+    int mngPort, int shufflePort, int webPort, String appName) {
     super("LlapDaemon");
 
     printAsciiArt();
@@ -158,11 +157,9 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
 
     this.maxJvmMemory = getTotalHeapSize();
     this.llapIoEnabled = ioEnabled;
-    Preconditions.checkArgument(headRoomBytes < executorMemoryBytes, "LLAP daemon headroom size should be less " +
-      "than daemon max memory size. headRoomBytes: " + headRoomBytes + " executorMemoryBytes: " + executorMemoryBytes);
-    final long minHeadRoomBytes = (long) (executorMemoryBytes * MIN_HEADROOM_PERCENT);
-    final long headroom = headRoomBytes < minHeadRoomBytes ? minHeadRoomBytes : headRoomBytes;
-    this.executorMemoryPerInstance = executorMemoryBytes - headroom;
+
+    long xmxHeadRoomBytes = determineXmxHeadroom(daemonConf, executorMemoryBytes, maxJvmMemory);
+    this.executorMemoryPerInstance = executorMemoryBytes - xmxHeadRoomBytes;
     this.ioMemoryPerInstance = ioMemoryBytes;
     this.numExecutors = numExecutors;
     this.localDirs = localDirs;
@@ -173,11 +170,14 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     boolean enablePreemption = HiveConf.getBoolVar(
         daemonConf, ConfVars.LLAP_DAEMON_TASK_SCHEDULER_ENABLE_PREEMPTION);
     LOG.warn("Attempting to start LlapDaemonConf with the following configuration: " +
-        "maxJvmMemory=" + maxJvmMemory + " (" + LlapUtil.humanReadableByteCount(maxJvmMemory) + ")" +
+        "maxJvmMemory=" + maxJvmMemory + " ("
+          + LlapUtil.humanReadableByteCount(maxJvmMemory) + ")" +
         ", requestedExecutorMemory=" + executorMemoryBytes +
         " (" + LlapUtil.humanReadableByteCount(executorMemoryBytes) + ")" +
-        ", llapIoCacheSize=" + ioMemoryBytes + " (" + LlapUtil.humanReadableByteCount(ioMemoryBytes) + ")" +
-        ", headRoomMemory=" + headroom + " (" + LlapUtil.humanReadableByteCount(headroom) + ")" +
+        ", llapIoCacheSize=" + ioMemoryBytes + " ("
+          + LlapUtil.humanReadableByteCount(ioMemoryBytes) + ")" +
+        ", xmxHeadRoomMemory=" + xmxHeadRoomBytes + " ("
+          + LlapUtil.humanReadableByteCount(xmxHeadRoomBytes) + ")" +
         ", adjustedExecutorMemory=" + executorMemoryPerInstance +
         " (" + LlapUtil.humanReadableByteCount(executorMemoryPerInstance) + ")" +
         ", numExecutors=" + numExecutors +
@@ -293,6 +293,30 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
     addIfService(amReporter);
   }
 
+  private static long determineXmxHeadroom(
+      Configuration daemonConf, long executorMemoryBytes, long maxJvmMemory) {
+    String headroomStr = HiveConf.getVar(daemonConf, ConfVars.LLAP_DAEMON_XMX_HEADROOM).trim();
+    long xmxHeadRoomBytes = Long.MAX_VALUE;
+    try {
+      if (headroomStr.endsWith("%")) {
+        long percentage = Integer.parseInt(headroomStr.substring(0, headroomStr.length() - 1));
+        Preconditions.checkState(percentage >= 0 && percentage < 100,
+            "Headroom percentage should be in [0, 100) range; found " + headroomStr);
+        xmxHeadRoomBytes = maxJvmMemory * percentage / 100L;
+      } else {
+        xmxHeadRoomBytes = HiveConf.toSizeBytes(headroomStr);
+      }
+    } catch (NumberFormatException ex) {
+      throw new RuntimeException("Invalid headroom configuration " + headroomStr);
+    }
+
+    Preconditions.checkArgument(xmxHeadRoomBytes < executorMemoryBytes,
+        "LLAP daemon headroom size should be less than daemon max memory size. headRoomBytes: "
+          + xmxHeadRoomBytes + " executorMemoryBytes: " + executorMemoryBytes + " (derived from "
+          + headroomStr + " out of xmx of " + maxJvmMemory + ")");
+    return xmxHeadRoomBytes;
+  }
+
   private static void initializeLogging(final Configuration conf) {
     long start = System.currentTimeMillis();
     URL llap_l4j2 = LlapDaemon.class.getClassLoader().getResource(
@@ -467,15 +491,14 @@ public class LlapDaemon extends CompositeService implements ContainerRunner, Lla
       int webPort = HiveConf.getIntVar(daemonConf, ConfVars.LLAP_DAEMON_WEB_PORT);
       long executorMemoryBytes = HiveConf.getIntVar(
           daemonConf, ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB) * 1024l * 1024l;
-      long headroomBytes = HiveConf.getIntVar(
-        daemonConf, ConfVars.LLAP_DAEMON_HEADROOM_MEMORY_PER_INSTANCE_MB) * 1024l * 1024l;
       long ioMemoryBytes = HiveConf.getSizeVar(daemonConf, ConfVars.LLAP_IO_MEMORY_MAX_SIZE);
       boolean isDirectCache = HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_ALLOCATOR_DIRECT);
       boolean isLlapIo = HiveConf.getBoolVar(daemonConf, HiveConf.ConfVars.LLAP_IO_ENABLED, true);
+
       LlapDaemon.initializeLogging(daemonConf);
       llapDaemon = new LlapDaemon(daemonConf, numExecutors, executorMemoryBytes, isLlapIo,
           isDirectCache, ioMemoryBytes, localDirs, rpcPort, mngPort, shufflePort, webPort,
-          appName, headroomBytes);
+          appName);
 
       LOG.info("Adding shutdown hook for LlapDaemon");
       ShutdownHookManager.addShutdownHook(new CompositeServiceShutdownHook(llapDaemon), 1);

http://git-wip-us.apache.org/repos/asf/hive/blob/de532b1f/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 66648b6..8a378ef 100644
--- a/llap-server/src/main/resources/package.py
+++ b/llap-server/src/main/resources/package.py
@@ -20,17 +20,15 @@ class LlapResource(object):
 		# convert to Mb
 		self.cache = config["hive.llap.io.memory.size"] / (1024*1024.0)
 		self.direct = config["hive.llap.io.allocator.direct"]
-		self.min_mb = -1
 		self.min_cores = -1
 		# compute heap + cache as final Xmx
 		h = self.memory 
 		if (not self.direct):
 			h += self.cache
 		if size == -1:
-			c = min(h*1.2, h + 1024) # + 1024 or 20%
-			c += (self.direct and self.cache) or 0
-			if self.min_mb > 0:
-				c = c + c%self.min_mb
+			print "Cannot determine the container size"
+			sys.exit(1)
+			return
 		else:
 			# do not mess with user input
 			c = size

http://git-wip-us.apache.org/repos/asf/hive/blob/de532b1f/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
----------------------------------------------------------------------
diff --git a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
index a9b23b6..06f6dac 100644
--- a/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
+++ b/llap-server/src/test/org/apache/hadoop/hive/llap/daemon/MiniLlapCluster.java
@@ -164,7 +164,7 @@ public class MiniLlapCluster extends AbstractService {
     LOG.info("Initializing {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);
     for (int i = 0 ;i < numInstances ; i++) {
       llapDaemons[i] = new LlapDaemon(conf, numExecutorsPerService, execBytesPerService, llapIoEnabled,
-          ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort, webPort, clusterNameTrimmed, 0);
+          ioIsDirect, ioBytesPerService, localDirs, rpcPort, mngPort, shufflePort, webPort, clusterNameTrimmed);
       llapDaemons[i].init(new Configuration(conf));
     }
     LOG.info("Initialized {} llap instances for MiniLlapCluster with name={}", numInstances, clusterNameTrimmed);


[20/50] [abbrv] hive git commit: HIVE-15973 : Make interval_arithmetic.q test robust

Posted by se...@apache.org.
HIVE-15973 : Make interval_arithmetic.q test robust


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

Branch: refs/heads/hive-14535
Commit: 1e00fb357c305b29cada4819793eb9ad5e63cb55
Parents: 54977d2
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Sun Feb 19 17:47:58 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Sun Feb 19 17:47:58 2017 -0800

----------------------------------------------------------------------
 .../test/queries/clientpositive/interval_arithmetic.q   |  4 ++--
 .../results/clientpositive/interval_arithmetic.q.out    | 12 ++++++------
 2 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1e00fb35/ql/src/test/queries/clientpositive/interval_arithmetic.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/interval_arithmetic.q b/ql/src/test/queries/clientpositive/interval_arithmetic.q
index 445cdfe..ae14579 100644
--- a/ql/src/test/queries/clientpositive/interval_arithmetic.q
+++ b/ql/src/test/queries/clientpositive/interval_arithmetic.q
@@ -160,6 +160,6 @@ from interval_arithmetic_1
 limit 2;
 
 explain
-select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1;
-select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1;
+select date '2016-11-08' + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1;
+select date '2016-11-08' + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1;
 drop table interval_arithmetic_1;

http://git-wip-us.apache.org/repos/asf/hive/blob/1e00fb35/ql/src/test/results/clientpositive/interval_arithmetic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/interval_arithmetic.q.out b/ql/src/test/results/clientpositive/interval_arithmetic.q.out
index 039d3e1..eba97fa 100644
--- a/ql/src/test/results/clientpositive/interval_arithmetic.q.out
+++ b/ql/src/test/results/clientpositive/interval_arithmetic.q.out
@@ -607,10 +607,10 @@ POSTHOOK: Input: default@interval_arithmetic_1
 109 20:30:40.246913578	89 02:14:26.000000000
 109 20:30:40.246913578	89 02:14:26.000000000
 PREHOOK: query: explain
-select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
+select date '2016-11-08' + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
 PREHOOK: type: QUERY
 POSTHOOK: query: explain
-select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
+select date '2016-11-08' + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
@@ -624,7 +624,7 @@ STAGE PLANS:
           alias: interval_arithmetic_1
           Statistics: Num rows: 12288 Data size: 326837 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: 2017-02-20 03:04:00.0 (type: timestamp)
+            expressions: 2016-11-11 03:04:00.0 (type: timestamp)
             outputColumnNames: _col0
             Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
             Limit
@@ -632,15 +632,15 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
               ListSink
 
-PREHOOK: query: select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
+PREHOOK: query: select date '2016-11-08' + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-POSTHOOK: query: select current_date + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
+POSTHOOK: query: select date '2016-11-08' + interval '1 2:02:00' day to second + interval '2' day + interval '1' hour + interval '1' minute + interval '60' second from interval_arithmetic_1 limit 1
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-2017-02-20 03:04:00
+2016-11-11 03:04:00
 PREHOOK: query: drop table interval_arithmetic_1
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@interval_arithmetic_1


[37/50] [abbrv] hive git commit: HIVE-15991 : Flaky Test: TestEncryptedHDFSCliDriver encryption_join_with_different_encryption_keys (Sahil Takiar via Ashutosh Chauhan) Signed-off-by: Ashutosh Chauhan

Posted by se...@apache.org.
HIVE-15991 : Flaky Test: TestEncryptedHDFSCliDriver encryption_join_with_different_encryption_keys (Sahil Takiar via Ashutosh Chauhan)
Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/hive-14535
Commit: ede8a5585d90f0306cc67e65e2f472c1fe351d12
Parents: 806d6e1
Author: Sahil Takiar <ta...@gmail.com>
Authored: Wed Feb 22 11:35:20 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Wed Feb 22 11:35:20 2017 -0800

----------------------------------------------------------------------
 ...cryption_join_with_different_encryption_keys.q.out | 14 ++++++--------
 1 file changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ede8a558/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
index 7a91538..54f6b27 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
@@ -1,12 +1,6 @@
-PREHOOK: query: --SORT_QUERY_RESULTS
-
--- Java JCE must be installed in order to hava a key length of 256 bits
-DROP TABLE IF EXISTS table_key_1 PURGE
+PREHOOK: query: DROP TABLE IF EXISTS table_key_1 PURGE
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: --SORT_QUERY_RESULTS
-
--- Java JCE must be installed in order to hava a key length of 256 bits
-DROP TABLE IF EXISTS table_key_1 PURGE
+POSTHOOK: query: DROP TABLE IF EXISTS table_key_1 PURGE
 POSTHOOK: type: DROPTABLE
 #### A masked pattern was here ####
 PREHOOK: type: CREATETABLE
@@ -121,6 +115,7 @@ STAGE PLANS:
             properties:
               COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
               bucket_count -1
+              column.name.delimiter ,
               columns key,value
               columns.comments 
               columns.types int:string
@@ -141,6 +136,7 @@ STAGE PLANS:
               properties:
                 COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
                 bucket_count -1
+                column.name.delimiter ,
                 columns key,value
                 columns.comments 
                 columns.types int:string
@@ -165,6 +161,7 @@ STAGE PLANS:
             properties:
               COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
               bucket_count -1
+              column.name.delimiter ,
               columns key,value
               columns.comments 
               columns.types int:string
@@ -185,6 +182,7 @@ STAGE PLANS:
               properties:
                 COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
                 bucket_count -1
+                column.name.delimiter ,
                 columns key,value
                 columns.comments 
                 columns.types int:string


[12/50] [abbrv] hive git commit: HIVE-15489: Alternatively use table scan stats for HoS (Chao Sun, reviewed by Xuefu Zhang)

Posted by se...@apache.org.
HIVE-15489: Alternatively use table scan stats for HoS (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/368d916b
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/368d916b
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/368d916b

Branch: refs/heads/hive-14535
Commit: 368d916b369f1adc58da884463b1dedb8c010616
Parents: bba1818
Author: Chao Sun <su...@apache.org>
Authored: Thu Jan 19 16:42:49 2017 -0800
Committer: Chao Sun <su...@apache.org>
Committed: Fri Feb 17 12:06:48 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   3 +
 .../test/resources/testconfiguration.properties |   3 +-
 .../hadoop/hive/ql/exec/OperatorUtils.java      |  34 +++
 .../SparkRemoveDynamicPruningBySize.java        |   4 +-
 .../optimizer/spark/SparkMapJoinOptimizer.java  |  34 ++-
 .../hive/ql/parse/spark/GenSparkUtils.java      |  36 +--
 .../hive/ql/parse/spark/SparkCompiler.java      |   3 +-
 .../spark_use_file_size_for_mapjoin.q           |  30 +++
 .../spark/spark_use_file_size_for_mapjoin.q.out | 257 +++++++++++++++++++
 9 files changed, 364 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/368d916b/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 0bff243..1af59ba 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3193,6 +3193,9 @@ public class HiveConf extends Configuration {
             Constants.LLAP_LOGGER_NAME_CONSOLE),
         "logger used for llap-daemons."),
 
+    SPARK_USE_FILE_SIZE_FOR_MAPJOIN("hive.spark.use.file.size.for.mapjoin", false,
+        "If this is set to true, mapjoin optimization in Hive/Spark will use source file sizes associated"
+            + "with TableScan operator on the root of operator tree, instead of using operator statistics."),
     SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout",
       "60s", new TimeValidator(TimeUnit.SECONDS),
       "Timeout for requests from Hive client to remote Spark driver."),

http://git-wip-us.apache.org/repos/asf/hive/blob/368d916b/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index e8db920..7c54275 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1481,7 +1481,8 @@ spark.query.files=add_part_multiple.q, \
 spark.only.query.files=spark_combine_equivalent_work.q,\
   spark_dynamic_partition_pruning.q,\
   spark_dynamic_partition_pruning_2.q,\
-  spark_vectorized_dynamic_partition_pruning.q
+  spark_vectorized_dynamic_partition_pruning.q,\
+  spark_use_file_size_for_mapjoin.q
 
 miniSparkOnYarn.query.files=auto_sortmerge_join_16.q,\
   bucket4.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/368d916b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
index d294e25..5bbfe12 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
@@ -346,4 +346,38 @@ public class OperatorUtils {
       }
     }
   }
+
+  /**
+   * Given the input operator 'op', walk up the operator tree from 'op', and collect all the
+   * roots that can be reached from it. The results are stored in 'roots'.
+   */
+  public static void findRoots(Operator<?> op, Collection<Operator<?>> roots) {
+    List<Operator<?>> parents = op.getParentOperators();
+    if (parents == null || parents.isEmpty()) {
+      roots.add(op);
+      return;
+    }
+    for (Operator<?> p : parents) {
+      findRoots(p, roots);
+    }
+  }
+
+  /**
+   * Remove the branch that contains the specified operator. Do nothing if there's no branching,
+   * i.e. all the upstream operators have only one child.
+   */
+  public static void removeBranch(Operator<?> op) {
+    Operator<?> child = op;
+    Operator<?> curr = op;
+
+    while (curr.getChildOperators().size() <= 1) {
+      child = curr;
+      if (curr.getParentOperators() == null || curr.getParentOperators().isEmpty()) {
+        return;
+      }
+      curr = curr.getParentOperators().get(0);
+    }
+
+    curr.removeChild(child);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/368d916b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruningBySize.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruningBySize.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruningBySize.java
index c41a0c8..26a1088 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruningBySize.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruningBySize.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer;
 
 import java.util.Stack;
 
+import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -28,7 +29,6 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.parse.spark.GenSparkUtils;
 import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext;
 import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
 
@@ -54,7 +54,7 @@ public class SparkRemoveDynamicPruningBySize implements NodeProcessor {
 
     if (desc.getStatistics().getDataSize() > context.getConf()
         .getLongVar(ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE)) {
-      GenSparkUtils.removeBranch(op);
+      OperatorUtils.removeBranch(op);
       // at this point we've found the fork in the op pipeline that has the pruning as a child plan.
       LOG.info("Disabling dynamic pruning for: "
           + desc.getTableScan().getName()

http://git-wip-us.apache.org/repos/asf/hive/blob/368d916b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
index 7faff88..d8f37ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
@@ -24,6 +24,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Stack;
 
+import org.apache.hadoop.hive.ql.exec.OperatorUtils;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -191,12 +193,40 @@ public class SparkMapJoinOptimizer implements NodeProcessor {
     int pos = 0;
 
     // bigTableFound means we've encountered a table that's bigger than the
-    // max. This table is either the the big table or we cannot convert.
+    // max. This table is either the big table or we cannot convert.
     boolean bigTableFound = false;
+    boolean useTsStats = context.getConf().getBoolean(HiveConf.ConfVars.SPARK_USE_FILE_SIZE_FOR_MAPJOIN.varname, false);
+    boolean hasUpstreamSinks = false;
 
+    // Check whether there's any upstream RS.
+    // If so, don't use TS stats because they could be inaccurate.
     for (Operator<? extends OperatorDesc> parentOp : joinOp.getParentOperators()) {
+      Set<ReduceSinkOperator> parentSinks =
+          OperatorUtils.findOperatorsUpstream(parentOp, ReduceSinkOperator.class);
+      parentSinks.remove(parentOp);
+      if (!parentSinks.isEmpty()) {
+        hasUpstreamSinks = true;
+      }
+    }
+
+    // If we are using TS stats and this JOIN has at least one upstream RS, disable MapJoin conversion.
+    if (useTsStats && hasUpstreamSinks) {
+      return new long[]{-1, 0, 0};
+    }
+
+    for (Operator<? extends OperatorDesc> parentOp : joinOp.getParentOperators()) {
+      Statistics currInputStat;
+      if (useTsStats) {
+        currInputStat = new Statistics();
+        // Find all root TSs and add up all data sizes
+        // Not adding other stats (e.g., # of rows, col stats) since only data size is used here
+        for (TableScanOperator root : OperatorUtils.findOperatorsUpstream(parentOp, TableScanOperator.class)) {
+          currInputStat.addToDataSize(root.getStatistics().getDataSize());
+        }
+      } else {
+         currInputStat = parentOp.getStatistics();
+      }
 
-      Statistics currInputStat = parentOp.getStatistics();
       if (currInputStat == null) {
         LOG.warn("Couldn't get statistics from: " + parentOp);
         return new long[]{-1, 0, 0};

http://git-wip-us.apache.org/repos/asf/hive/blob/368d916b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
index 7b2b3c0..36bde30 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
@@ -573,7 +574,7 @@ public class GenSparkUtils {
    */
   public BaseWork getEnclosingWork(Operator<?> op, GenSparkProcContext procCtx) {
     List<Operator<?>> ops = new ArrayList<Operator<?>>();
-    findRoots(op, ops);
+    OperatorUtils.findRoots(op, ops);
     for (Operator<?> r : ops) {
       BaseWork work = procCtx.rootToWorkMap.get(r);
       if (work != null) {
@@ -582,37 +583,4 @@ public class GenSparkUtils {
     }
     return null;
   }
-
-  /*
-   * findRoots returns all root operators (in ops) that result in operator op
-   */
-  private void findRoots(Operator<?> op, List<Operator<?>> ops) {
-    List<Operator<?>> parents = op.getParentOperators();
-    if (parents == null || parents.isEmpty()) {
-      ops.add(op);
-      return;
-    }
-    for (Operator<?> p : parents) {
-      findRoots(p, ops);
-    }
-  }
-
-  /**
-   * Remove the branch that contains the specified operator. Do nothing if there's no branching,
-   * i.e. all the upstream operators have only one child.
-   */
-  public static void removeBranch(Operator<?> op) {
-    Operator<?> child = op;
-    Operator<?> curr = op;
-
-    while (curr.getChildOperators().size() <= 1) {
-      child = curr;
-      if (curr.getParentOperators() == null || curr.getParentOperators().isEmpty()) {
-        return;
-      }
-      curr = curr.getParentOperators().get(0);
-    }
-
-    curr.removeChild(child);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/368d916b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
index 71528e8..c4b1640 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
@@ -169,7 +170,7 @@ public class SparkCompiler extends TaskCompiler {
       return;
     }
 
-    GenSparkUtils.removeBranch(toRemove);
+    OperatorUtils.removeBranch(toRemove);
     // at this point we've found the fork in the op pipeline that has the pruning as a child plan.
     LOG.info("Disabling dynamic pruning for: "
         + toRemove.getConf().getTableScan().toString() + ". Needed to break cyclic dependency");

http://git-wip-us.apache.org/repos/asf/hive/blob/368d916b/ql/src/test/queries/clientpositive/spark_use_file_size_for_mapjoin.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/spark_use_file_size_for_mapjoin.q b/ql/src/test/queries/clientpositive/spark_use_file_size_for_mapjoin.q
new file mode 100644
index 0000000..b623b83
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/spark_use_file_size_for_mapjoin.q
@@ -0,0 +1,30 @@
+set hive.mapred.mode=nonstrict;
+set hive.auto.convert.join=true;
+set hive.spark.use.file.size.for.mapjoin=true;
+set hive.auto.convert.join.noconditionaltask.size=4000;
+
+EXPLAIN
+SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97;
+
+SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97;
+
+set hive.auto.convert.join.noconditionaltask.size=8000;
+
+-- This is copied from auto_join2. Without the configuration both joins are mapjoins,
+-- but with the configuration on, Hive should not turn the second join into mapjoin since it
+-- has a upstream reduce sink.
+
+CREATE TABLE dest(key INT, value STRING) STORED AS TEXTFILE;
+
+EXPLAIN
+FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key)
+INSERT OVERWRITE TABLE dest SELECT src1.key, src3.value;
+
+FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key)
+INSERT OVERWRITE TABLE dest SELECT src1.key, src3.value;
+
+SELECT sum(hash(dest.key,dest.value)) FROM dest;

http://git-wip-us.apache.org/repos/asf/hive/blob/368d916b/ql/src/test/results/clientpositive/spark/spark_use_file_size_for_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/spark_use_file_size_for_mapjoin.q.out b/ql/src/test/results/clientpositive/spark/spark_use_file_size_for_mapjoin.q.out
new file mode 100644
index 0000000..9044140
--- /dev/null
+++ b/ql/src/test/results/clientpositive/spark/spark_use_file_size_for_mapjoin.q.out
@@ -0,0 +1,257 @@
+PREHOOK: query: EXPLAIN
+SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Spark
+      Edges:
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src1
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToDouble(key) = 97.0) (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: src2
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToDouble(key) = 97.0) (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
+        Reducer 2 
+            Reduce Operator Tree:
+              Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string), _col2 (type: string)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.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 src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+97	val_97
+97	val_97
+97	val_97
+97	val_97
+PREHOOK: query: CREATE TABLE dest(key INT, value STRING) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@dest
+POSTHOOK: query: CREATE TABLE dest(key INT, value STRING) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@dest
+PREHOOK: query: EXPLAIN
+FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key)
+INSERT OVERWRITE TABLE dest SELECT src1.key, src3.value
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key)
+INSERT OVERWRITE TABLE dest SELECT src1.key, src3.value
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-3 is a root stage
+  Stage-1 depends on stages: Stage-3
+  Stage-0 depends on stages: Stage-1
+  Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-3
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: src2
+                  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)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                      Spark HashTable Sink Operator
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+      Edges:
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src1
+                  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)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0, _col1
+                        input vertices:
+                          1 Map 3
+                        Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: (UDFToDouble(_col0) + UDFToDouble(_col1)) (type: double)
+                          sort order: +
+                          Map-reduce partition columns: (UDFToDouble(_col0) + UDFToDouble(_col1)) (type: double)
+                          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: string)
+            Local Work:
+              Map Reduce Local Work
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: src3
+                  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: UDFToDouble(_col0) (type: double)
+                        sort order: +
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double)
+                        Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
+        Reducer 2 
+            Reduce Operator Tree:
+              Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 (UDFToDouble(_col0) + UDFToDouble(_col1)) (type: double)
+                  1 UDFToDouble(_col0) (type: double)
+                outputColumnNames: _col0, _col3
+                Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: UDFToInteger(_col0) (type: int), _col3 (type: string)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        name: default.dest
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: true
+          table:
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.dest
+
+  Stage: Stage-2
+    Stats-Aggr Operator
+
+PREHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key)
+INSERT OVERWRITE TABLE dest SELECT src1.key, src3.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@dest
+POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key)
+INSERT OVERWRITE TABLE dest SELECT src1.key, src3.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@dest
+POSTHOOK: Lineage: dest.key EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: dest.value SIMPLE [(src)src3.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: SELECT sum(hash(dest.key,dest.value)) FROM dest
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dest
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT sum(hash(dest.key,dest.value)) FROM dest
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dest
+#### A masked pattern was here ####
+33815990627


[07/50] [abbrv] hive git commit: HIVE-15936: ConcurrentModificationException in ATSHook (Daniel Dai, reviewed by Jason Dere)

Posted by se...@apache.org.
HIVE-15936: ConcurrentModificationException in ATSHook (Daniel Dai, reviewed by Jason Dere)


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

Branch: refs/heads/hive-14535
Commit: 3485d02cbfca5603e86e7b370da86b607003f337
Parents: 90688bd
Author: Daniel Dai <da...@hortonworks.com>
Authored: Fri Feb 17 00:22:54 2017 -0800
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Fri Feb 17 00:22:54 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/hooks/ATSHook.java     | 15 ++++++++++-----
 1 file changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3485d02c/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
index 55b922b..72a1acc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
@@ -143,6 +143,11 @@ public class ATSHook implements ExecuteWithHookContext {
     final QueryState queryState = hookContext.getQueryState();
     final String queryId = queryState.getQueryId();
 
+    final Map<String, Long> durations = new HashMap<String, Long>();
+    for (String key : hookContext.getPerfLogger().getEndTimes().keySet()) {
+      durations.put(key, hookContext.getPerfLogger().getDuration(key));
+    }
+
     try {
       setupAtsExecutor(conf);
 
@@ -203,10 +208,10 @@ public class ATSHook implements ExecuteWithHookContext {
                         tablesRead, tablesWritten, conf, llapId));
                 break;
               case POST_EXEC_HOOK:
-                fireAndForget(createPostHookEvent(queryId, currentTime, user, requestuser, true, opId, hookContext.getPerfLogger()));
+                fireAndForget(createPostHookEvent(queryId, currentTime, user, requestuser, true, opId, durations));
                 break;
               case ON_FAILURE_HOOK:
-                fireAndForget(createPostHookEvent(queryId, currentTime, user, requestuser , false, opId, hookContext.getPerfLogger()));
+                fireAndForget(createPostHookEvent(queryId, currentTime, user, requestuser , false, opId, durations));
                 break;
               default:
                 //ignore
@@ -325,7 +330,7 @@ public class ATSHook implements ExecuteWithHookContext {
   }
 
   TimelineEntity createPostHookEvent(String queryId, long stopTime, String user, String requestuser, boolean success,
-      String opId, PerfLogger perfLogger) throws Exception {
+      String opId, Map<String, Long> durations) throws Exception {
     LOG.info("Received post-hook notification for :" + queryId);
 
     TimelineEntity atsEntity = new TimelineEntity();
@@ -346,8 +351,8 @@ public class ATSHook implements ExecuteWithHookContext {
 
     // Perf times
     JSONObject perfObj = new JSONObject(new LinkedHashMap<>());
-    for (String key : perfLogger.getEndTimes().keySet()) {
-      perfObj.put(key, perfLogger.getDuration(key));
+    for (Map.Entry<String, Long> entry : durations.entrySet()) {
+      perfObj.put(entry.getKey(), entry.getValue());
     }
     atsEntity.addOtherInfo(OtherInfoTypes.PERF.name(), perfObj.toString());
 


[19/50] [abbrv] hive git commit: HIVE-15941: Fix o.a.h.hive.ql.exec.tez.TezTask compilation issue with tez master (Rajesh Balamohan, reviewed by Siddharth Seth)

Posted by se...@apache.org.
HIVE-15941: Fix o.a.h.hive.ql.exec.tez.TezTask compilation issue with tez master (Rajesh Balamohan, reviewed by Siddharth Seth)


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

Branch: refs/heads/hive-14535
Commit: 54977d2f05fca63d1bcfebf6673f85a3f5c6f8ad
Parents: 0debf9f
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Mon Feb 20 05:53:41 2017 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Mon Feb 20 05:53:41 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/54977d2f/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index 69cbe0b..58f0b33 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -627,6 +627,17 @@ public class TezTask extends Task<TezWork> {
       dagClient.close(); // Don't sync.
     }
 
+    public String getDagIdentifierString() {
+      // TODO: Implement this when tez is upgraded. TEZ-3550
+      return null;
+    }
+
+    public String getSessionIdentifierString() {
+      // TODO: Implement this when tez is upgraded. TEZ-3550
+      return null;
+    }
+
+
     @Override
     public String getExecutionContext() {
       return dagClient.getExecutionContext(); // Don't sync.


[03/50] [abbrv] hive git commit: HIVE-15917 : incorrect error handling from BackgroundWork can cause beeline query to hang (Sergey Shelukhin, reviewed by Siddharth Seth)

Posted by se...@apache.org.
HIVE-15917 : incorrect error handling from BackgroundWork can cause beeline query to hang (Sergey Shelukhin, reviewed by Siddharth Seth)


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

Branch: refs/heads/hive-14535
Commit: e49a07426884d8494a37046a227ff4a77cf67f57
Parents: 60a36d1
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Feb 16 12:39:55 2017 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Feb 16 12:39:55 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hive/service/cli/CLIService.java | 45 +++++++++++---------
 .../hive/service/cli/operation/Operation.java   | 16 +------
 .../service/cli/operation/SQLOperation.java     |  4 +-
 3 files changed, 30 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e49a0742/service/src/java/org/apache/hive/service/cli/CLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/CLIService.java b/service/src/java/org/apache/hive/service/cli/CLIService.java
index b842f37..714b259 100644
--- a/service/src/java/org/apache/hive/service/cli/CLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/CLIService.java
@@ -404,7 +404,7 @@ public class CLIService extends CompositeService implements ICLIService {
    */
   @Override
   public OperationHandle getPrimaryKeys(SessionHandle sessionHandle,
-		  String catalog, String schema, String table)
+      String catalog, String schema, String table)
           throws HiveSQLException {
     OperationHandle opHandle = sessionManager.getSession(sessionHandle)
         .getPrimaryKeys(catalog, schema, table);
@@ -417,9 +417,9 @@ public class CLIService extends CompositeService implements ICLIService {
    */
   @Override
   public OperationHandle getCrossReference(SessionHandle sessionHandle,
-		  String primaryCatalog,
-	      String primarySchema, String primaryTable, String foreignCatalog,
-	      String foreignSchema, String foreignTable)
+      String primaryCatalog,
+        String primarySchema, String primaryTable, String foreignCatalog,
+        String foreignSchema, String foreignTable)
           throws HiveSQLException {
     OperationHandle opHandle = sessionManager.getSession(sessionHandle)
         .getCrossReference(primaryCatalog, primarySchema, primaryTable,
@@ -460,6 +460,8 @@ public class CLIService extends CompositeService implements ICLIService {
         // The background operation thread was cancelled
         LOG.trace(opHandle + ": The background operation was cancelled", e);
       } catch (ExecutionException e) {
+        // Note: Hive ops do not use the normal Future failure path, so this will not happen
+        //       in case of actual failure; the Future will just be done.
         // The background operation thread was aborted
         LOG.warn(opHandle + ": The background operation was aborted", e);
       } catch (InterruptedException e) {
@@ -473,23 +475,28 @@ public class CLIService extends CompositeService implements ICLIService {
     return opStatus;
   }
 
+  private static final long PROGRESS_MAX_WAIT_NS = 30 * 1000000000l;
   private JobProgressUpdate progressUpdateLog(boolean isProgressLogRequested, Operation operation) {
-    if (isProgressLogRequested && canProvideProgressLog()) {
-        if (OperationType.EXECUTE_STATEMENT.equals(operation.getType())) {
-          SessionState sessionState = operation.getParentSession().getSessionState();
-          try {
-            while (sessionState.getProgressMonitor() == null && !operation.isFinished()) {
-              Thread.sleep(10L); // sleep for 10 ms
-            }
-          } catch (InterruptedException e) {
-            LOG.warn("Error while getting progress update", e);
-          }
-          if (sessionState.getProgressMonitor() != null) {
-            return new JobProgressUpdate(sessionState.getProgressMonitor());
-          }
-        }
+    if (!isProgressLogRequested || !canProvideProgressLog()
+        || !OperationType.EXECUTE_STATEMENT.equals(operation.getType())) {
+      return new JobProgressUpdate(ProgressMonitor.NULL);
+    }
+    
+    SessionState sessionState = operation.getParentSession().getSessionState();
+    long startTime = System.nanoTime();
+    int timeOutMs = 8;
+    try {
+      while (sessionState.getProgressMonitor() == null && !operation.isDone()) {
+        long remainingMs = (PROGRESS_MAX_WAIT_NS - (System.nanoTime() - startTime)) / 1000000l;
+        if (remainingMs <= 0) return new JobProgressUpdate(ProgressMonitor.NULL);
+        Thread.sleep(Math.min(remainingMs, timeOutMs));
+        timeOutMs <<= 1;
+      }
+    } catch (InterruptedException e) {
+      LOG.warn("Error while getting progress update", e);
     }
-    return new JobProgressUpdate(ProgressMonitor.NULL);
+    ProgressMonitor pm = sessionState.getProgressMonitor();
+    return new JobProgressUpdate(pm != null ? pm : ProgressMonitor.NULL);
   }
 
   private boolean canProvideProgressLog() {

http://git-wip-us.apache.org/repos/asf/hive/blob/e49a0742/service/src/java/org/apache/hive/service/cli/operation/Operation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/Operation.java b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
index 2039946..11a820f 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/Operation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/Operation.java
@@ -206,20 +206,8 @@ public abstract class Operation {
     this.lastAccessTime = System.currentTimeMillis();
   }
 
-  public boolean isRunning() {
-    return OperationState.RUNNING.equals(state);
-  }
-
-  public boolean isFinished() {
-    return OperationState.FINISHED.equals(state);
-  }
-
-  public boolean isCanceled() {
-    return OperationState.CANCELED.equals(state);
-  }
-
-  public boolean isFailed() {
-    return OperationState.ERROR.equals(state);
+  public boolean isDone() {
+    return state.isTerminal();
   }
 
   protected void createOperationLog() {

http://git-wip-us.apache.org/repos/asf/hive/blob/e49a0742/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 668b4b7..7dde7bf 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
@@ -346,6 +346,7 @@ public class SQLOperation extends ExecuteStatementOperation {
             }
             runQuery();
           } catch (HiveSQLException e) {
+            // TODO: why do we invent our own error path op top of the one from Future.get?
             setOperationException(e);
             LOG.error("Error running hive query: ", e);
           } finally {
@@ -361,8 +362,7 @@ public class SQLOperation extends ExecuteStatementOperation {
       } catch (Exception e) {
         setOperationException(new HiveSQLException(e));
         LOG.error("Error running hive query as user : " + currentUGI.getShortUserName(), e);
-      }
-      finally {
+      } finally {
         /**
          * We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup
          * when this thread is garbage collected later.


[23/50] [abbrv] hive git commit: HIVE-15972: Runtime filtering not vectorizing for decimal/timestamp/char/varchar (Jason Dere, reviewed by Matt McCline)

Posted by se...@apache.org.
HIVE-15972: Runtime filtering not vectorizing for decimal/timestamp/char/varchar (Jason Dere, reviewed by Matt McCline)


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

Branch: refs/heads/hive-14535
Commit: 016afe0d69f3a90290e3a127149430ad6d4c603f
Parents: 08ca7b2
Author: Jason Dere <jd...@hortonworks.com>
Authored: Mon Feb 20 13:03:06 2017 -0800
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Mon Feb 20 13:03:06 2017 -0800

----------------------------------------------------------------------
 .../FilterColumnBetweenDynamicValue.txt         |   2 +
 .../vectorized_dynamic_semijoin_reduction2.q    |  17 +-
 ...vectorized_dynamic_semijoin_reduction2.q.out | 428 ++++++++++++++++++-
 .../apache/hadoop/hive/tools/GenVectorCode.java |   8 +-
 4 files changed, 446 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/016afe0d/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
index 1aee9b3..9d5432f 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterColumnBetweenDynamicValue.txt
@@ -27,7 +27,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.sql.Timestamp;
+import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
 
 public class <ClassName> extends Filter<TypeName>ColumnBetween {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/016afe0d/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q b/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
index 446407d..be8e4af 100644
--- a/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
+++ b/ql/src/test/queries/clientpositive/vectorized_dynamic_semijoin_reduction2.q
@@ -18,7 +18,10 @@ create table dsrv2_big stored as orc as
   cast(L_PARTKEY as decimal(10,1)) as partkey_decimal,
   cast(L_PARTKEY as double) as partkey_double,
   cast(l_shipdate as date) as shipdate_date,
-  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts
+  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts,
+  cast(l_shipdate as string) as shipdate_string,
+  cast(l_shipdate as char(10)) as shipdate_char,
+  cast(l_shipdate as varchar(10)) as shipdate_varchar
   from lineitem;
 create table dsrv2_small stored as orc as select * from dsrv2_big limit 20;
 analyze table dsrv2_big compute statistics;
@@ -46,5 +49,17 @@ select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_date = b.ship
 EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts);
 select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_ts = b.shipdate_ts);
 
+-- single key (string)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_string = b.shipdate_string);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_string = b.shipdate_string);
+
+-- single key (char)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_char = b.shipdate_char);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_char = b.shipdate_char);
+
+-- single key (varchar)
+EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_varchar = b.shipdate_varchar);
+select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_varchar = b.shipdate_varchar);
+
 drop table dsrv2_big;
 drop table dsrv2_small;

http://git-wip-us.apache.org/repos/asf/hive/blob/016afe0d/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
index 27d8152..062fef6 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
@@ -4,7 +4,10 @@ PREHOOK: query: create table dsrv2_big stored as orc as
   cast(L_PARTKEY as decimal(10,1)) as partkey_decimal,
   cast(L_PARTKEY as double) as partkey_double,
   cast(l_shipdate as date) as shipdate_date,
-  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts
+  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts,
+  cast(l_shipdate as string) as shipdate_string,
+  cast(l_shipdate as char(10)) as shipdate_char,
+  cast(l_shipdate as varchar(10)) as shipdate_varchar
   from lineitem
 PREHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: Input: default@lineitem
@@ -16,7 +19,10 @@ POSTHOOK: query: create table dsrv2_big stored as orc as
   cast(L_PARTKEY as decimal(10,1)) as partkey_decimal,
   cast(L_PARTKEY as double) as partkey_double,
   cast(l_shipdate as date) as shipdate_date,
-  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts
+  cast(cast(l_shipdate as date) as timestamp) as shipdate_ts,
+  cast(l_shipdate as string) as shipdate_string,
+  cast(l_shipdate as char(10)) as shipdate_char,
+  cast(l_shipdate as varchar(10)) as shipdate_varchar
   from lineitem
 POSTHOOK: type: CREATETABLE_AS_SELECT
 POSTHOOK: Input: default@lineitem
@@ -25,8 +31,11 @@ POSTHOOK: Output: default@dsrv2_big
 POSTHOOK: Lineage: dsrv2_big.partkey_bigint EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_partkey, type:int, comment:null), ]
 POSTHOOK: Lineage: dsrv2_big.partkey_decimal EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_partkey, type:int, comment:null), ]
 POSTHOOK: Lineage: dsrv2_big.partkey_double EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_partkey, type:int, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.shipdate_char EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_shipdate, type:string, comment:null), ]
 POSTHOOK: Lineage: dsrv2_big.shipdate_date EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_shipdate, type:string, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.shipdate_string SIMPLE [(lineitem)lineitem.FieldSchema(name:l_shipdate, type:string, comment:null), ]
 POSTHOOK: Lineage: dsrv2_big.shipdate_ts EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_shipdate, type:string, comment:null), ]
+POSTHOOK: Lineage: dsrv2_big.shipdate_varchar EXPRESSION [(lineitem)lineitem.FieldSchema(name:l_shipdate, type:string, comment:null), ]
 PREHOOK: query: create table dsrv2_small stored as orc as select * from dsrv2_big limit 20
 PREHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: Input: default@dsrv2_big
@@ -40,8 +49,11 @@ POSTHOOK: Output: default@dsrv2_small
 POSTHOOK: Lineage: dsrv2_small.partkey_bigint SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:partkey_bigint, type:bigint, comment:null), ]
 POSTHOOK: Lineage: dsrv2_small.partkey_decimal SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:partkey_decimal, type:decimal(10,1), comment:null), ]
 POSTHOOK: Lineage: dsrv2_small.partkey_double SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:partkey_double, type:double, comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.shipdate_char SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:shipdate_char, type:char(10), comment:null), ]
 POSTHOOK: Lineage: dsrv2_small.shipdate_date SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:shipdate_date, type:date, comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.shipdate_string SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:shipdate_string, type:string, comment:null), ]
 POSTHOOK: Lineage: dsrv2_small.shipdate_ts SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:shipdate_ts, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: dsrv2_small.shipdate_varchar SIMPLE [(dsrv2_big)dsrv2_big.FieldSchema(name:shipdate_varchar, type:varchar(10), comment:null), ]
 PREHOOK: query: analyze table dsrv2_big compute statistics
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dsrv2_big
@@ -247,7 +259,7 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: decimal(10,1))
                         Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 4 
             Map Operator Tree:
@@ -655,7 +667,7 @@ STAGE PLANS:
                         sort order: +
                         Map-reduce partition columns: _col0 (type: timestamp)
                         Statistics: Num rows: 100 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 4 
             Map Operator Tree:
@@ -754,6 +766,414 @@ POSTHOOK: Input: default@dsrv2_big
 POSTHOOK: Input: default@dsrv2_small
 #### A masked pattern was here ####
 23
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_string = b.shipdate_string)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_string = b.shipdate_string)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (shipdate_string is not null and shipdate_string BETWEEN DynamicValue(RS_7_b_shipdate_string_min) AND DynamicValue(RS_7_b_shipdate_string_max) and in_bloom_filter(shipdate_string, DynamicValue(RS_7_b_shipdate_string_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (shipdate_string is not null and shipdate_string BETWEEN DynamicValue(RS_7_b_shipdate_string_min) AND DynamicValue(RS_7_b_shipdate_string_max) and in_bloom_filter(shipdate_string, DynamicValue(RS_7_b_shipdate_string_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_string (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: shipdate_string is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: shipdate_string is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_string (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=15)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                Statistics: Num rows: 20 Data size: 160 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)
+        Reducer 3 
+            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
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=15)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_string = b.shipdate_string)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_string = b.shipdate_string)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+23
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_char = b.shipdate_char)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_char = b.shipdate_char)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (shipdate_char is not null and shipdate_char BETWEEN DynamicValue(RS_7_b_shipdate_char_min) AND DynamicValue(RS_7_b_shipdate_char_max) and in_bloom_filter(shipdate_char, DynamicValue(RS_7_b_shipdate_char_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (shipdate_char is not null and shipdate_char BETWEEN DynamicValue(RS_7_b_shipdate_char_min) AND DynamicValue(RS_7_b_shipdate_char_max) and in_bloom_filter(shipdate_char, DynamicValue(RS_7_b_shipdate_char_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_char (type: char(10))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(10))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: char(10))
+                        Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: shipdate_char is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: shipdate_char is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_char (type: char(10))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(10))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: char(10))
+                        Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: char(10))
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=15)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: char(10)), _col1 (type: char(10)), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: char(10))
+                  1 _col0 (type: char(10))
+                Statistics: Num rows: 20 Data size: 160 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)
+        Reducer 3 
+            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
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=15)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: char(10)), _col1 (type: char(10)), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_char = b.shipdate_char)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_char = b.shipdate_char)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+23
+PREHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_varchar = b.shipdate_varchar)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_varchar = b.shipdate_varchar)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (shipdate_varchar is not null and shipdate_varchar BETWEEN DynamicValue(RS_7_b_shipdate_varchar_min) AND DynamicValue(RS_7_b_shipdate_varchar_max) and in_bloom_filter(shipdate_varchar, DynamicValue(RS_7_b_shipdate_varchar_bloom_filter))) (type: boolean)
+                  Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (shipdate_varchar is not null and shipdate_varchar BETWEEN DynamicValue(RS_7_b_shipdate_varchar_min) AND DynamicValue(RS_7_b_shipdate_varchar_max) and in_bloom_filter(shipdate_varchar, DynamicValue(RS_7_b_shipdate_varchar_bloom_filter))) (type: boolean)
+                    Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_varchar (type: varchar(10))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: varchar(10))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: varchar(10))
+                        Statistics: Num rows: 100 Data size: 9400 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: shipdate_varchar is not null (type: boolean)
+                  Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: shipdate_varchar is not null (type: boolean)
+                    Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: shipdate_varchar (type: varchar(10))
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: varchar(10))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: varchar(10))
+                        Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: varchar(10))
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 20 Data size: 1880 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), bloom_filter(_col0, expectedEntries=15)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: varchar(10)), _col1 (type: varchar(10)), _col2 (type: binary)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: varchar(10))
+                  1 _col0 (type: varchar(10))
+                Statistics: Num rows: 20 Data size: 160 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)
+        Reducer 3 
+            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
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), bloom_filter(VALUE._col2, expectedEntries=15)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: varchar(10)), _col1 (type: varchar(10)), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_varchar = b.shipdate_varchar)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dsrv2_big
+PREHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from dsrv2_big a join dsrv2_small b on (a.shipdate_varchar = b.shipdate_varchar)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dsrv2_big
+POSTHOOK: Input: default@dsrv2_small
+#### A masked pattern was here ####
+23
 PREHOOK: query: drop table dsrv2_big
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@dsrv2_big

http://git-wip-us.apache.org/repos/asf/hive/blob/016afe0d/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
----------------------------------------------------------------------
diff --git a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
index 55cfb7b..926321e 100644
--- a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
+++ b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
@@ -1418,7 +1418,7 @@ public class GenVectorCode extends Task {
       getValueMethod = "";
       conversionMethod = "";
     } else if (operandType.equals("decimal")) {
-      defaultValue = "null";
+      defaultValue = "HiveDecimal.ZERO";
       vectorType = "HiveDecimal";
       getPrimitiveMethod = "getHiveDecimal";
       getValueMethod = "";
@@ -1430,13 +1430,13 @@ public class GenVectorCode extends Task {
       getValueMethod = ".getBytes()";
       conversionMethod = "";
     } else if (operandType.equals("char")) {
-      defaultValue = "null";
+      defaultValue = "new HiveChar(\"\", 1)";
       vectorType = "byte[]";
       getPrimitiveMethod = "getHiveChar";
       getValueMethod = ".getStrippedValue().getBytes()";  // Does vectorization use stripped char values?
       conversionMethod = "";
     } else if (operandType.equals("varchar")) {
-      defaultValue = "null";
+      defaultValue = "new HiveVarchar(\"\", 1)";
       vectorType = "byte[]";
       getPrimitiveMethod = "getHiveVarchar";
       getValueMethod = ".getValue().getBytes()";
@@ -1450,7 +1450,7 @@ public class GenVectorCode extends Task {
       // Special case - Date requires its own specific BetweenDynamicValue class, but derives from FilterLongColumnBetween
       typeName = "Long";
     } else if (operandType.equals("timestamp")) {
-      defaultValue = "null";
+      defaultValue = "new Timestamp(0)";
       vectorType = "Timestamp";
       getPrimitiveMethod = "getTimestamp";
       getValueMethod = "";


[21/50] [abbrv] hive git commit: HIVE-15891 : Detect query rewrite scenario for UPDATE/DELETE/MERGE and fail fast (Wei Zheng, reviewed by Eugene Koifman)

Posted by se...@apache.org.
HIVE-15891 : Detect query rewrite scenario for UPDATE/DELETE/MERGE and fail fast (Wei Zheng, reviewed by Eugene Koifman)


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

Branch: refs/heads/hive-14535
Commit: 091ac8e05940ce327d068cee77817bbe428797e2
Parents: 1e00fb3
Author: Wei Zheng <we...@apache.org>
Authored: Sun Feb 19 18:17:10 2017 -0800
Committer: Wei Zheng <we...@apache.org>
Committed: Sun Feb 19 18:17:10 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/Context.java | 12 +++++++++++
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |  3 +++
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  5 +++++
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |  1 +
 .../clientnegative/masking_acid_delete.q        | 10 ++++++++++
 .../queries/clientnegative/masking_acid_merge.q | 15 ++++++++++++++
 .../clientnegative/masking_acid_update.q        | 10 ++++++++++
 .../clientnegative/masking_acid_delete.q.out    | 13 ++++++++++++
 .../clientnegative/masking_acid_merge.q.out     | 21 ++++++++++++++++++++
 .../clientnegative/masking_acid_update.q.out    | 13 ++++++++++++
 10 files changed, 103 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/ql/src/java/org/apache/hadoop/hive/ql/Context.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
index fae2a12..d1d2789 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
@@ -125,6 +125,10 @@ public class Context {
   private Heartbeater heartbeater;
 
   private boolean skipTableMasking;
+
+  // Identify whether the query involves an UPDATE, DELETE or MERGE
+  private boolean isUpdateDeleteMerge;
+
   /**
    * This determines the prefix of the
    * {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.Phase1Ctx#dest}
@@ -955,4 +959,12 @@ public class Context {
     opContext = new CompilationOpContext();
     sequencer = new AtomicInteger();
   }
+
+  public boolean getIsUpdateDeleteMerge() {
+    return isUpdateDeleteMerge;
+  }
+
+  public void setIsUpdateDeleteMerge(boolean isUpdate) {
+    this.isUpdateDeleteMerge = isUpdate;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/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 6013218..2ffc130 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -398,6 +398,9 @@ public enum ErrorMsg {
       "time."),
   DISTINCT_NOT_SUPPORTED(10285, "Distinct keyword is not support in current context"),
   NONACID_COMPACTION_NOT_SUPPORTED(10286, "Compaction is not allowed on non-ACID table {0}.{1}", true),
+  MASKING_FILTERING_ON_ACID_NOT_SUPPORTED(10287,
+      "Detected {0}.{1} has row masking/column filtering enabled, " +
+      "which is not supported for query involving ACID operations", true),
 
   UPDATEDELETE_PARSE_ERROR(10290, "Encountered parse error while parsing rewritten merge/update or " +
       "delete query"),

http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/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 f053093..9c37af8 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
@@ -10820,6 +10820,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         .applyRowFilterAndColumnMasking(basicPrivObjs);
     if (needRewritePrivObjs != null && !needRewritePrivObjs.isEmpty()) {
       for (HivePrivilegeObject privObj : needRewritePrivObjs) {
+        // We don't support masking/filtering against ACID query at the moment
+        if (ctx.getIsUpdateDeleteMerge()) {
+          throw new SemanticException(ErrorMsg.MASKING_FILTERING_ON_ACID_NOT_SUPPORTED,
+              privObj.getDbname(), privObj.getObjectName());
+        }
         MaskAndFilterInfo info = basicInfos.get(privObj);
         String replacementText = tableMask.create(privObj, info);
         if (replacementText != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
index 725f2ce..865c03a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
@@ -293,6 +293,7 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
       HiveConf.setVar(conf, HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict");
       rewrittenCtx = new Context(conf);
       rewrittenCtx.setExplainConfig(ctx.getExplainConfig());
+      rewrittenCtx.setIsUpdateDeleteMerge(true);
     } catch (IOException e) {
       throw new SemanticException(ErrorMsg.UPDATEDELETE_IO_ERROR.getMsg());
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/ql/src/test/queries/clientnegative/masking_acid_delete.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/masking_acid_delete.q b/ql/src/test/queries/clientnegative/masking_acid_delete.q
new file mode 100644
index 0000000..2785d7a
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/masking_acid_delete.q
@@ -0,0 +1,10 @@
+set hive.mapred.mode=nonstrict;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+create table masking_test (key int, value string)
+clustered by (value) into 2 buckets stored as orc
+tblproperties ("transactional"="true");
+
+delete from masking_test where value='ddd';

http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/ql/src/test/queries/clientnegative/masking_acid_merge.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/masking_acid_merge.q b/ql/src/test/queries/clientnegative/masking_acid_merge.q
new file mode 100644
index 0000000..50471e0
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/masking_acid_merge.q
@@ -0,0 +1,15 @@
+set hive.mapred.mode=nonstrict;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+create table nonacid (key int, value string) stored as orc;
+
+create table masking_test (key int, value string)
+clustered by (value) into 2 buckets stored as orc
+tblproperties ("transactional"="true");
+
+MERGE INTO masking_test as t using nonacid as s ON t.key = s.key
+WHEN MATCHED AND s.key < 5 THEN DELETE
+WHEN MATCHED AND s.key < 3 THEN UPDATE set key = 1
+WHEN NOT MATCHED THEN INSERT VALUES (s.key, s.value);

http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/ql/src/test/queries/clientnegative/masking_acid_update.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/masking_acid_update.q b/ql/src/test/queries/clientnegative/masking_acid_update.q
new file mode 100644
index 0000000..586d20b
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/masking_acid_update.q
@@ -0,0 +1,10 @@
+set hive.mapred.mode=nonstrict;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+create table masking_test (key int, value string)
+clustered by (value) into 2 buckets stored as orc
+tblproperties ("transactional"="true");
+
+update masking_test set key=1 where value='ddd';

http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/ql/src/test/results/clientnegative/masking_acid_delete.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/masking_acid_delete.q.out b/ql/src/test/results/clientnegative/masking_acid_delete.q.out
new file mode 100644
index 0000000..021f2cd
--- /dev/null
+++ b/ql/src/test/results/clientnegative/masking_acid_delete.q.out
@@ -0,0 +1,13 @@
+PREHOOK: query: create table masking_test (key int, value string)
+clustered by (value) into 2 buckets stored as orc
+tblproperties ("transactional"="true")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@masking_test
+POSTHOOK: query: create table masking_test (key int, value string)
+clustered by (value) into 2 buckets stored as orc
+tblproperties ("transactional"="true")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@masking_test
+FAILED: SemanticException [Error 10287]: Detected default.masking_test has row masking/column filtering enabled, which is not supported for query involving ACID operations

http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/ql/src/test/results/clientnegative/masking_acid_merge.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/masking_acid_merge.q.out b/ql/src/test/results/clientnegative/masking_acid_merge.q.out
new file mode 100644
index 0000000..917c44a
--- /dev/null
+++ b/ql/src/test/results/clientnegative/masking_acid_merge.q.out
@@ -0,0 +1,21 @@
+PREHOOK: query: create table nonacid (key int, value string) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@nonacid
+POSTHOOK: query: create table nonacid (key int, value string) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@nonacid
+PREHOOK: query: create table masking_test (key int, value string)
+clustered by (value) into 2 buckets stored as orc
+tblproperties ("transactional"="true")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@masking_test
+POSTHOOK: query: create table masking_test (key int, value string)
+clustered by (value) into 2 buckets stored as orc
+tblproperties ("transactional"="true")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@masking_test
+FAILED: SemanticException [Error 10287]: Detected default.masking_test has row masking/column filtering enabled, which is not supported for query involving ACID operations

http://git-wip-us.apache.org/repos/asf/hive/blob/091ac8e0/ql/src/test/results/clientnegative/masking_acid_update.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/masking_acid_update.q.out b/ql/src/test/results/clientnegative/masking_acid_update.q.out
new file mode 100644
index 0000000..021f2cd
--- /dev/null
+++ b/ql/src/test/results/clientnegative/masking_acid_update.q.out
@@ -0,0 +1,13 @@
+PREHOOK: query: create table masking_test (key int, value string)
+clustered by (value) into 2 buckets stored as orc
+tblproperties ("transactional"="true")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@masking_test
+POSTHOOK: query: create table masking_test (key int, value string)
+clustered by (value) into 2 buckets stored as orc
+tblproperties ("transactional"="true")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@masking_test
+FAILED: SemanticException [Error 10287]: Detected default.masking_test has row masking/column filtering enabled, which is not supported for query involving ACID operations


[33/50] [abbrv] hive git commit: HIVE-15570. Improved messaging when llap is down, ignore consistent splits config in non llap mode. (Zhiyuan Yang, reviewed by Sergey Shelukhin, Siddharth Seth)

Posted by se...@apache.org.
HIVE-15570. Improved messaging when llap is down, ignore consistent splits config in non llap mode. (Zhiyuan Yang, reviewed by Sergey Shelukhin, Siddharth Seth)


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

Branch: refs/heads/hive-14535
Commit: dc0938c42f6c9a42adb3fcbb391fb759a3bb0072
Parents: 78e4bb7
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Feb 21 22:52:54 2017 -0800
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Feb 21 22:52:54 2017 -0800

----------------------------------------------------------------------
 common/src/java/org/apache/hadoop/hive/conf/HiveConf.java | 9 +++++----
 ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java | 6 +++++-
 2 files changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/dc0938c4/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 4faaa8b..7c88f4f 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3162,10 +3162,11 @@ public class HiveConf extends Configuration {
       "llap.daemon.service.port"),
     LLAP_DAEMON_WEB_SSL("hive.llap.daemon.web.ssl", false,
       "Whether LLAP daemon web UI should use SSL.", "llap.daemon.service.ssl"),
-    LLAP_CLIENT_CONSISTENT_SPLITS("hive.llap.client.consistent.splits",
-        false,
-        "Whether to setup split locations to match nodes on which llap daemons are running," +
-            " instead of using the locations provided by the split itself"),
+    LLAP_CLIENT_CONSISTENT_SPLITS("hive.llap.client.consistent.splits", false,
+        "Whether to setup split locations to match nodes on which llap daemons are running, " +
+        "instead of using the locations provided by the split itself. If there is no llap daemon " +
+        "running, fall back to locations provided by the split. This is effective only if " +
+        "hive.execution.mode is llap"),
     LLAP_VALIDATE_ACLS("hive.llap.validate.acls", true,
         "Whether LLAP should reject permissive ACLs in some cases (e.g. its own management\n" +
         "protocol or ZK paths), similar to how ssh refuses a key with bad access permissions."),

http://git-wip-us.apache.org/repos/asf/hive/blob/dc0938c4/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
index d691e18..2b57d90 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/Utils.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -34,7 +35,8 @@ public class Utils {
   public static SplitLocationProvider getSplitLocationProvider(Configuration conf, Logger LOG) throws
       IOException {
     boolean useCustomLocations =
-        HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS);
+        HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_EXECUTION_MODE).equals("llap")
+        && HiveConf.getBoolVar(conf, HiveConf.ConfVars.LLAP_CLIENT_CONSISTENT_SPLITS);
     SplitLocationProvider splitLocationProvider;
     LOG.info("SplitGenerator using llap affinitized locations: " + useCustomLocations);
     if (useCustomLocations) {
@@ -43,6 +45,8 @@ public class Utils {
 
       Collection<ServiceInstance> serviceInstances =
           serviceRegistry.getInstances().getAllInstancesOrdered(true);
+      Preconditions.checkArgument(!serviceInstances.isEmpty(),
+          "No running LLAP daemons! Please check LLAP service status and zookeeper configuration");
       ArrayList<String> locations = new ArrayList<>(serviceInstances.size());
       for (ServiceInstance serviceInstance : serviceInstances) {
         if (LOG.isDebugEnabled()) {


[34/50] [abbrv] hive git commit: HIVE-15990: Always initialize connection properties in DruidSerDe (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-15990: Always initialize connection properties in DruidSerDe (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/8973d2c6
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8973d2c6
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8973d2c6

Branch: refs/heads/hive-14535
Commit: 8973d2c66394ed25b1baa20df3920870ae9b053c
Parents: dc0938c
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Mon Feb 20 17:32:46 2017 +0000
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Feb 22 10:16:50 2017 +0000

----------------------------------------------------------------------
 .../hadoop/hive/druid/serde/DruidSerDe.java       | 18 ++++++++----------
 1 file changed, 8 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8973d2c6/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
index 4235e89..bbe29b6 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
@@ -100,18 +100,21 @@ public class DruidSerDe extends AbstractSerDe {
 
   protected static final Logger LOG = LoggerFactory.getLogger(DruidSerDe.class);
 
-  private String[] columns;
-
-  private PrimitiveTypeInfo[] types;
-
   private int numConnection;
-
   private Period readTimeout;
 
+  private String[] columns;
+  private PrimitiveTypeInfo[] types;
   private ObjectInspector inspector;
 
   @Override
   public void initialize(Configuration configuration, Properties properties) throws SerDeException {
+    // Init connection properties
+    numConnection = HiveConf
+          .getIntVar(configuration, HiveConf.ConfVars.HIVE_DRUID_NUM_HTTP_CONNECTION);
+    readTimeout = new Period(
+          HiveConf.getVar(configuration, HiveConf.ConfVars.HIVE_DRUID_HTTP_READ_TIMEOUT));
+
     final List<String> columnNames = new ArrayList<>();
     final List<PrimitiveTypeInfo> columnTypes = new ArrayList<>();
     List<ObjectInspector> inspectors = new ArrayList<>();
@@ -173,11 +176,6 @@ public class DruidSerDe extends AbstractSerDe {
           throw new SerDeException("Druid broker address not specified in configuration");
         }
 
-        numConnection = HiveConf
-              .getIntVar(configuration, HiveConf.ConfVars.HIVE_DRUID_NUM_HTTP_CONNECTION);
-        readTimeout = new Period(
-              HiveConf.getVar(configuration, HiveConf.ConfVars.HIVE_DRUID_HTTP_READ_TIMEOUT));
-
         // Infer schema
         SegmentAnalysis schemaInfo;
         try {


[32/50] [abbrv] hive git commit: HIVE-15938 : position alias in order by fails for union queries (Sergey Shelukhin, reviewed by Ashutosh Chauhan, Pengcheng Xiong)

Posted by se...@apache.org.
HIVE-15938 : position alias in order by fails for union queries (Sergey Shelukhin, reviewed by Ashutosh Chauhan, Pengcheng Xiong)


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

Branch: refs/heads/hive-14535
Commit: 78e4bb79a2f9e74acb8144db1854e5b9ad369f0f
Parents: ffe7357
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Feb 21 17:15:14 2017 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Feb 21 17:41:44 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   6 +-
 .../metadata/HiveMaterializedViewsRegistry.java |   3 +-
 .../calcite/translator/ASTBuilder.java          |  38 +--
 .../index/RewriteParseContextGenerator.java     |   4 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  51 ++-
 .../ql/parse/ColumnStatsAutoGatherContext.java  |   4 +-
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |   4 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  11 +-
 .../apache/hadoop/hive/ql/parse/ParseUtils.java | 190 +++++++++++-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   7 +-
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |   5 +-
 .../hadoop/hive/ql/tools/LineageInfo.java       |   5 +-
 .../ql/parse/TestMacroSemanticAnalyzer.java     |   4 +-
 .../parse/TestUpdateDeleteSemanticAnalyzer.java |   4 +-
 .../authorization/AuthorizationTestUtil.java    |   2 +-
 .../queries/clientpositive/union_pos_alias.q    |  30 ++
 .../clientpositive/constant_prop_1.q.out        |   4 +-
 .../clientpositive/union_pos_alias.q.out        | 308 +++++++++++++++++++
 18 files changed, 611 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/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 2423471..592b1f1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -118,7 +118,6 @@ import org.apache.hadoop.hive.ql.session.OperationLog.LoggingLevel;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.hadoop.hive.serde2.ByteStream;
-import org.apache.hadoop.hive.serde2.thrift.ThriftJDBCBinarySerDe;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.JobClient;
@@ -466,9 +465,7 @@ public class Driver implements CommandProcessor {
       ctx.setHDFSCleanup(true);
 
       perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.PARSE);
-      ParseDriver pd = new ParseDriver();
-      ASTNode tree = pd.parse(command, ctx);
-      tree = ParseUtils.findRootNonNullToken(tree);
+      ASTNode tree = ParseUtils.parse(command, ctx);
       perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PARSE);
 
       // Trigger query hook before compilation
@@ -646,6 +643,7 @@ public class Driver implements CommandProcessor {
     }
   }
 
+
   private int handleInterruption(String msg) {
     SQLState = "HY008";  //SQLState for cancel operation
     errorMessage = "FAILED: command has been interrupted: " + msg;

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java
index 89c87cd..1d78b4c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveMaterializedViewsRegistry.java
@@ -328,8 +328,7 @@ public final class HiveMaterializedViewsRegistry {
 
   private static RelNode parseQuery(String viewQuery) {
     try {
-      final ParseDriver pd = new ParseDriver();
-      final ASTNode node = ParseUtils.findRootNonNullToken(pd.parse(viewQuery));
+      final ASTNode node = ParseUtils.parse(viewQuery);
       final QueryState qs = new QueryState(SessionState.get().getConf());
       CalcitePlanner analyzer = new CalcitePlanner(qs);
       analyzer.initCtx(new Context(SessionState.get().getConf()));

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
index e36e1bd..0dc0c24 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
@@ -38,19 +38,19 @@ import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.ParseDriver;
 import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
 
-class ASTBuilder {
+public class ASTBuilder {
 
-  static ASTBuilder construct(int tokenType, String text) {
+  public static ASTBuilder construct(int tokenType, String text) {
     ASTBuilder b = new ASTBuilder();
     b.curr = createAST(tokenType, text);
     return b;
   }
 
-  static ASTNode createAST(int tokenType, String text) {
+  public static ASTNode createAST(int tokenType, String text) {
     return (ASTNode) ParseDriver.adaptor.create(tokenType, text);
   }
 
-  static ASTNode destNode() {
+  public static ASTNode destNode() {
     return ASTBuilder
         .construct(HiveParser.TOK_DESTINATION, "TOK_DESTINATION")
         .add(
@@ -58,7 +58,7 @@ class ASTBuilder {
                 "TOK_TMP_FILE")).node();
   }
 
-  static ASTNode table(RelNode scan) {
+  public static ASTNode table(RelNode scan) {
     HiveTableScan hts;
     if (scan instanceof DruidQuery) {
       hts = (HiveTableScan) ((DruidQuery)scan).getTableScan();
@@ -102,7 +102,7 @@ class ASTBuilder {
     return b.node();
   }
 
-  static ASTNode join(ASTNode left, ASTNode right, JoinRelType joinType, ASTNode cond,
+  public static ASTNode join(ASTNode left, ASTNode right, JoinRelType joinType, ASTNode cond,
       boolean semiJoin) {
     ASTBuilder b = null;
 
@@ -129,12 +129,12 @@ class ASTBuilder {
     return b.node();
   }
 
-  static ASTNode subQuery(ASTNode qry, String alias) {
+  public static ASTNode subQuery(ASTNode qry, String alias) {
     return ASTBuilder.construct(HiveParser.TOK_SUBQUERY, "TOK_SUBQUERY").add(qry)
         .add(HiveParser.Identifier, alias).node();
   }
 
-  static ASTNode qualifiedName(String tableName, String colName) {
+  public static ASTNode qualifiedName(String tableName, String colName) {
     ASTBuilder b = ASTBuilder
         .construct(HiveParser.DOT, ".")
         .add(
@@ -143,36 +143,36 @@ class ASTBuilder {
     return b.node();
   }
 
-  static ASTNode unqualifiedName(String colName) {
+  public static ASTNode unqualifiedName(String colName) {
     ASTBuilder b = ASTBuilder.construct(HiveParser.TOK_TABLE_OR_COL, "TOK_TABLE_OR_COL").add(
         HiveParser.Identifier, colName);
     return b.node();
   }
 
-  static ASTNode where(ASTNode cond) {
+  public static ASTNode where(ASTNode cond) {
     return ASTBuilder.construct(HiveParser.TOK_WHERE, "TOK_WHERE").add(cond).node();
   }
 
-  static ASTNode having(ASTNode cond) {
+  public static ASTNode having(ASTNode cond) {
     return ASTBuilder.construct(HiveParser.TOK_HAVING, "TOK_HAVING").add(cond).node();
   }
 
-  static ASTNode limit(Object offset, Object limit) {
+  public static ASTNode limit(Object offset, Object limit) {
     return ASTBuilder.construct(HiveParser.TOK_LIMIT, "TOK_LIMIT")
         .add(HiveParser.Number, offset.toString())
         .add(HiveParser.Number, limit.toString()).node();
   }
 
-  static ASTNode selectExpr(ASTNode expr, String alias) {
+  public static ASTNode selectExpr(ASTNode expr, String alias) {
     return ASTBuilder.construct(HiveParser.TOK_SELEXPR, "TOK_SELEXPR").add(expr)
         .add(HiveParser.Identifier, alias).node();
   }
 
-  static ASTNode literal(RexLiteral literal) {
+  public static ASTNode literal(RexLiteral literal) {
     return literal(literal, false);
   }
 
-  static ASTNode literal(RexLiteral literal, boolean useTypeQualInLiteral) {
+  public static ASTNode literal(RexLiteral literal, boolean useTypeQualInLiteral) {
     Object val = null;
     int type = 0;
     SqlTypeName sqlType = literal.getType().getSqlTypeName();
@@ -328,21 +328,21 @@ class ASTBuilder {
 
   ASTNode curr;
 
-  ASTNode node() {
+  public ASTNode node() {
     return curr;
   }
 
-  ASTBuilder add(int tokenType, String text) {
+  public ASTBuilder add(int tokenType, String text) {
     ParseDriver.adaptor.addChild(curr, createAST(tokenType, text));
     return this;
   }
 
-  ASTBuilder add(ASTBuilder b) {
+  public ASTBuilder add(ASTBuilder b) {
     ParseDriver.adaptor.addChild(curr, b.curr);
     return this;
   }
 
-  ASTBuilder add(ASTNode n) {
+  public ASTBuilder add(ASTNode n) {
     if (n != null) {
       ParseDriver.adaptor.addChild(curr, n);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteParseContextGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteParseContextGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteParseContextGenerator.java
index 340d29a..5659a72 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteParseContextGenerator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteParseContextGenerator.java
@@ -62,9 +62,7 @@ public final class RewriteParseContextGenerator {
     Operator<? extends OperatorDesc> operatorTree;
     try {
       Context ctx = new Context(queryState.getConf());
-      ParseDriver pd = new ParseDriver();
-      ASTNode tree = pd.parse(command, ctx);
-      tree = ParseUtils.findRootNonNullToken(tree);
+      ASTNode tree = ParseUtils.parse(command, ctx);
 
       BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, tree);
       assert(sem instanceof SemanticAnalyzer);

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/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 10f16ca..21bf020 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
@@ -973,17 +973,18 @@ public class CalcitePlanner extends SemanticAnalyzer {
     return newAst;
   }
 
-  /**
-   * Performs breadth-first search of the AST for a nested set of tokens. Tokens
-   * don't have to be each others' direct children, they can be separated by
-   * layers of other tokens. For each token in the list, the first one found is
-   * matched and there's no backtracking; thus, if AST has multiple instances of
-   * some token, of which only one matches, it is not guaranteed to be found. We
-   * use this for simple things. Not thread-safe - reuses searchQueue.
-   */
-  static class ASTSearcher {
+
+  public static class ASTSearcher {
     private final LinkedList<ASTNode> searchQueue = new LinkedList<ASTNode>();
 
+    /**
+     * Performs breadth-first search of the AST for a nested set of tokens. Tokens
+     * don't have to be each others' direct children, they can be separated by
+     * layers of other tokens. For each token in the list, the first one found is
+     * matched and there's no backtracking; thus, if AST has multiple instances of
+     * some token, of which only one matches, it is not guaranteed to be found. We
+     * use this for simple things. Not thread-safe - reuses searchQueue.
+     */
     public ASTNode simpleBreadthFirstSearch(ASTNode ast, int... tokens) {
       searchQueue.clear();
       searchQueue.add(ast);
@@ -1007,6 +1008,38 @@ public class CalcitePlanner extends SemanticAnalyzer {
       }
       return null;
     }
+ 
+    public ASTNode depthFirstSearch(ASTNode ast, int token) {
+      searchQueue.clear();
+      searchQueue.add(ast);
+      while (!searchQueue.isEmpty()) {
+        ASTNode next = searchQueue.poll();
+        if (next.getType() == token) return next;
+        for (int j = 0; j < next.getChildCount(); ++j) {
+          searchQueue.add((ASTNode) next.getChild(j));
+        }
+      }
+      return null;
+    }
+
+    public ASTNode simpleBreadthFirstSearchAny(ASTNode ast, int... tokens) {
+      searchQueue.clear();
+      searchQueue.add(ast);
+      while (!searchQueue.isEmpty()) {
+        ASTNode next = searchQueue.poll();
+        for (int i = 0; i < tokens.length; ++i) {
+          if (next.getType() == tokens[i]) return next;
+        }
+        for (int i = 0; i < next.getChildCount(); ++i) {
+          searchQueue.add((ASTNode) next.getChild(i));
+        }
+      }
+      return null;
+    }
+
+    public void reset() {
+      searchQueue.clear();
+    }
   }
 
   private static void replaceASTChild(ASTNode child, ASTNode newChild) {

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
index 80e62c1..3b719af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
@@ -132,9 +132,7 @@ public class ColumnStatsAutoGatherContext {
     //0. initialization
     Context ctx = new Context(conf);
     ctx.setExplainConfig(origCtx.getExplainConfig());
-    ParseDriver pd = new ParseDriver();
-    ASTNode tree = pd.parse(analyzeCommand, ctx);
-    tree = ParseUtils.findRootNonNullToken(tree);
+    ASTNode tree = ParseUtils.parse(analyzeCommand, ctx);
 
     //1. get the ColumnStatsSemanticAnalyzer
     BaseSemanticAnalyzer baseSem = SemanticAnalyzerFactory.get(new QueryState(conf), tree);

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
index ff07b42..93b8183 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
@@ -294,14 +294,12 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
       throw new SemanticException(ErrorMsg.COLUMNSTATSCOLLECTOR_IO_ERROR.getMsg());
     }
     ctx.setCmd(rewrittenQuery);
-    ParseDriver pd = new ParseDriver();
 
     try {
-      rewrittenTree = pd.parse(rewrittenQuery, ctx);
+      rewrittenTree = ParseUtils.parse(rewrittenQuery, ctx);
     } catch (ParseException e) {
       throw new SemanticException(ErrorMsg.COLUMNSTATSCOLLECTOR_PARSE_ERROR.getMsg());
     }
-    rewrittenTree = ParseUtils.findRootNonNullToken(rewrittenTree);
     return rewrittenTree;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index eb81393..b4b5bfb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -42,6 +42,7 @@ TOK_SUBQUERY;
 TOK_INSERT_INTO;
 TOK_DESTINATION;
 TOK_ALLCOLREF;
+TOK_SETCOLREF;
 TOK_TABLE_OR_COL;
 TOK_FUNCTION;
 TOK_FUNCTIONDI;
@@ -2442,7 +2443,7 @@ fromStatement
 	        )
 	       ^(TOK_INSERT 
 	          ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-	          ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF))
+	          ^(TOK_SELECT ^(TOK_SELEXPR TOK_SETCOLREF))
 	        )
 	      )
     -> {$fromStatement.tree}
@@ -2526,7 +2527,7 @@ selectStatement
           )
           ^(TOK_INSERT
              ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-             ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF))
+             ^(TOK_SELECT ^(TOK_SELEXPR TOK_SETCOLREF))
              $o? $c? $d? $sort? $l?
           )
       )
@@ -2545,7 +2546,7 @@ setOpSelectStatement[CommonTree t]
           )
           ^(TOK_INSERT
              ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-             ^(TOK_SELECTDI ^(TOK_SELEXPR TOK_ALLCOLREF))
+             ^(TOK_SELECTDI ^(TOK_SELEXPR TOK_SETCOLREF))
           )
        )
    -> {$setOpSelectStatement.tree != null && ((CommonTree)u.getTree()).getType()!=HiveParser.TOK_UNIONDISTINCT}?
@@ -2560,7 +2561,7 @@ setOpSelectStatement[CommonTree t]
            )
           ^(TOK_INSERT
             ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-            ^(TOK_SELECTDI ^(TOK_SELEXPR TOK_ALLCOLREF))
+            ^(TOK_SELECTDI ^(TOK_SELEXPR TOK_SETCOLREF))
          )
        )
    -> ^($u {$t} $b)
@@ -2579,7 +2580,7 @@ setOpSelectStatement[CommonTree t]
           )
           ^(TOK_INSERT
              ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-             ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF))
+             ^(TOK_SELECT ^(TOK_SELEXPR TOK_SETCOLREF))
           )
        )
    -> {$setOpSelectStatement.tree}

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
index 943e6af..473a664 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
@@ -18,6 +18,15 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
+import org.apache.hadoop.hive.ql.Context;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.antlr.runtime.tree.CommonTree;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTBuilder;
+import org.apache.hadoop.hive.ql.parse.CalcitePlanner.ASTSearcher;
+
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -48,6 +57,26 @@ import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
  *
  */
 public final class ParseUtils {
+  /** Parses the Hive query. */
+  private static final Logger LOG = LoggerFactory.getLogger(ParseUtils.class);
+  public static ASTNode parse(String command) throws ParseException {
+    return parse(command, null);
+  }
+
+  /** Parses the Hive query. */
+  public static ASTNode parse(String command, Context ctx) throws ParseException {
+    return parse(command, ctx, true);
+  }
+
+  /** Parses the Hive query. */
+  public static ASTNode parse(
+      String command, Context ctx, boolean setTokenRewriteStream) throws ParseException {
+    ParseDriver pd = new ParseDriver();
+    ASTNode tree = pd.parse(command, ctx, setTokenRewriteStream);
+    tree = findRootNonNullToken(tree);
+    handleSetColRefs(tree);
+    return tree;
+  }
 
   /**
    * Tests whether the parse tree node is a join token.
@@ -77,7 +106,7 @@ public final class ParseUtils {
    *
    * @return node at which descent stopped
    */
-  public static ASTNode findRootNonNullToken(ASTNode tree) {
+  private static ASTNode findRootNonNullToken(ASTNode tree) {
     while ((tree.getToken() == null) && (tree.getChildCount() > 0)) {
       tree = (ASTNode) tree.getChild(0);
     }
@@ -311,4 +340,163 @@ public final class ParseUtils {
 
       return stack.empty() && otherStack.empty();
     }
+
+
+    private static void handleSetColRefs(ASTNode tree) {
+      CalcitePlanner.ASTSearcher astSearcher = new CalcitePlanner.ASTSearcher();
+      while (true) {
+        astSearcher.reset();
+        ASTNode setCols = astSearcher.depthFirstSearch(tree, HiveParser.TOK_SETCOLREF);
+        if (setCols == null) break;
+        processSetColsNode(setCols, astSearcher);
+      }
+    }
+
+    /**
+     * Replaces a spurious TOK_SETCOLREF added by parser with column names referring to the query
+     * in e.g. a union. This is to maintain the expectations that some code, like order by position
+     * alias, might have about not having ALLCOLREF. If it cannot find the columns with confidence
+     * it will just replace SETCOLREF with ALLCOLREF. Most of the cases where that happens are
+     * easy to work around in the query (e.g. by adding column aliases in the union).
+     * @param setCols TOK_SETCOLREF ASTNode.
+     * @param searcher AST searcher to reuse.
+     */
+    private static void processSetColsNode(ASTNode setCols, ASTSearcher searcher) {
+      searcher.reset();
+      CommonTree rootNode = setCols;
+      while (rootNode != null && rootNode.getType() != HiveParser.TOK_INSERT) {
+        rootNode = rootNode.parent;
+      }
+      if (rootNode == null || rootNode.parent == null) {
+        // Couldn't find the parent insert; replace with ALLCOLREF.
+        LOG.debug("Replacing SETCOLREF with ALLCOLREF because we couldn't find the root INSERT");
+        setCols.token.setType(HiveParser.TOK_ALLCOLREF);
+        return;
+      }
+      rootNode = rootNode.parent; // TOK_QUERY above insert
+      Tree fromNode = null;
+      for (int j = 0; j < rootNode.getChildCount(); ++j) {
+        Tree child = rootNode.getChild(j);
+        if (child.getType() == HiveParser.TOK_FROM) {
+          fromNode = child;
+          break;
+        }
+      }
+      if (!(fromNode instanceof ASTNode)) {
+        // Couldn't find the from that contains subquery; replace with ALLCOLREF.
+        LOG.debug("Replacing SETCOLREF with ALLCOLREF because we couldn't find the FROM");
+        setCols.token.setType(HiveParser.TOK_ALLCOLREF);
+        return;
+      }
+      // We are making what we are trying to do more explicit if there's a union alias; so
+      // that if we do something we didn't expect to do, it'd be more likely to fail.
+      String alias = null;
+      if (fromNode.getChildCount() > 0) {
+        Tree fromWhat = fromNode.getChild(0);
+        if (fromWhat.getType() == HiveParser.TOK_SUBQUERY && fromWhat.getChildCount() > 1) {
+          Tree child = fromWhat.getChild(fromWhat.getChildCount() - 1);
+          if (child.getType() == HiveParser.Identifier) {
+            alias = child.getText();
+          }
+        }
+      }
+      // We find the SELECT closest to the top. This assumes there's only one FROM or FROM-s
+      // are all equivalent (union case). Also, this assumption could be false for an already
+      // malformed query; we don't check for that here - it will fail later anyway.
+      // TODO: Maybe we should find ALL the SELECT-s not nested in another from, and compare.
+      ASTNode select = searcher.simpleBreadthFirstSearchAny((ASTNode)fromNode,
+          HiveParser.TOK_SELECT, HiveParser.TOK_SELECTDI);
+      if (select == null) {
+        // Couldn't find the from that contains subquery; replace with ALLCOLREF.
+        LOG.debug("Replacing SETCOLREF with ALLCOLREF because we couldn't find the SELECT");
+        setCols.token.setType(HiveParser.TOK_ALLCOLREF);
+        return;
+      }
+      // Found the proper columns.
+      List<ASTNode> newChildren = new ArrayList<>(select.getChildCount());
+      HashSet<String> aliases = new HashSet<>();
+      for (int i = 0; i < select.getChildCount(); ++i) {
+        Tree selExpr = select.getChild(i);
+        assert selExpr.getType() == HiveParser.TOK_SELEXPR;
+        assert selExpr.getChildCount() > 0;
+        // Examine the last child. It could be an alias.
+        Tree child = selExpr.getChild(selExpr.getChildCount() - 1);
+        switch (child.getType()) {
+        case HiveParser.TOK_SETCOLREF:
+          // We have a nested setcolref. Process that and start from scratch TODO: use stack?
+          processSetColsNode((ASTNode)child, searcher);
+          processSetColsNode(setCols, searcher);
+          return;
+        case HiveParser.TOK_ALLCOLREF:
+          // We should find an alias of this insert and do (alias).*. This however won't fix e.g.
+          // positional order by alias case, cause we'd still have a star on the top level. Bail.
+          LOG.debug("Replacing SETCOLREF with ALLCOLREF because of nested ALLCOLREF");
+          setCols.token.setType(HiveParser.TOK_ALLCOLREF);
+          return;
+        case HiveParser.TOK_TABLE_OR_COL:
+          Tree idChild = child.getChild(0);
+          assert idChild.getType() == HiveParser.Identifier : idChild;
+          if (!createChildColumnRef(idChild, alias, newChildren, aliases)) {
+            setCols.token.setType(HiveParser.TOK_ALLCOLREF);
+            return;
+          }
+          break;
+        case HiveParser.Identifier:
+          if (!createChildColumnRef(child, alias, newChildren, aliases)) {
+            setCols.token.setType(HiveParser.TOK_ALLCOLREF);
+            return;
+          }
+          break;
+        case HiveParser.DOT: {
+          Tree colChild = child.getChild(child.getChildCount() - 1);
+          assert colChild.getType() == HiveParser.Identifier : colChild;
+          if (!createChildColumnRef(colChild, alias, newChildren, aliases)) {
+            setCols.token.setType(HiveParser.TOK_ALLCOLREF);
+            return;
+          }
+          break;
+        }
+        default:
+          // Not really sure how to refer to this (or if we can).
+          // TODO: We could find a different from branch for the union, that might have an alias?
+          //       Or we could add an alias here to refer to, but that might break other branches.
+          LOG.debug("Replacing SETCOLREF with ALLCOLREF because of the nested node "
+              + child.getType() + " " + child.getText());
+          setCols.token.setType(HiveParser.TOK_ALLCOLREF);
+          return;
+        }
+      }
+      // Insert search in the beginning would have failed if these parents didn't exist.
+      ASTNode parent = (ASTNode)setCols.parent.parent;
+      int t = parent.getType();
+      assert t == HiveParser.TOK_SELECT || t == HiveParser.TOK_SELECTDI : t;
+      int ix = setCols.parent.childIndex;
+      parent.deleteChild(ix);
+      for (ASTNode node : newChildren) {
+        parent.insertChild(ix++, node);
+      }
+    }
+
+    private static boolean createChildColumnRef(Tree child, String alias,
+        List<ASTNode> newChildren, HashSet<String> aliases) {
+      String colAlias = child.getText();
+      if (!aliases.add(colAlias)) {
+        // TODO: if a side of the union has 2 columns with the same name, noone on the higher
+        //       level can refer to them. We could change the alias in the original node.
+        LOG.debug("Replacing SETCOLREF with ALLCOLREF because of duplicate alias " + colAlias);
+        return false;
+      }
+      ASTBuilder selExpr = ASTBuilder.construct(HiveParser.TOK_SELEXPR, "TOK_SELEXPR");
+      ASTBuilder toc = ASTBuilder.construct(HiveParser.TOK_TABLE_OR_COL, "TOK_TABLE_OR_COL");
+      ASTBuilder id = ASTBuilder.construct(HiveParser.Identifier, colAlias);
+      if (alias == null) {
+        selExpr = selExpr.add(toc.add(id));
+      } else {
+        ASTBuilder dot = ASTBuilder.construct(HiveParser.DOT, ".");
+        ASTBuilder aliasNode = ASTBuilder.construct(HiveParser.Identifier, alias);
+        selExpr = selExpr.add(dot.add(toc.add(aliasNode)).add(id));
+      }
+      newChildren.add(selExpr.node());
+      return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/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 9eafb0b..2430811 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
@@ -2429,7 +2429,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   private void replaceViewReferenceWithDefinition(QB qb, Table tab,
       String tab_name, String alias) throws SemanticException {
 
-    ParseDriver pd = new ParseDriver();
     ASTNode viewTree;
     final ASTNodeOrigin viewOrigin = new ASTNodeOrigin("VIEW", tab.getTableName(),
         tab.getViewExpandedText(), alias, qb.getParseInfo().getSrcForAlias(
@@ -2438,8 +2437,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       String viewText = tab.getViewExpandedText();
       // Reparse text, passing null for context to avoid clobbering
       // the top-level token stream.
-      ASTNode tree = pd.parse(viewText, ctx, false);
-      tree = ParseUtils.findRootNonNullToken(tree);
+      ASTNode tree = ParseUtils.parse(viewText, ctx, false);
       viewTree = tree;
       Dispatcher nodeOriginDispatcher = new Dispatcher() {
         @Override
@@ -10885,11 +10883,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       // check if we need to ctx.setCmd(rewrittenQuery);
       ParseDriver pd = new ParseDriver();
       try {
-        rewrittenTree = pd.parse(rewrittenQuery);
+        rewrittenTree = ParseUtils.parse(rewrittenQuery);
       } catch (ParseException e) {
         throw new SemanticException(e);
       }
-      rewrittenTree = ParseUtils.findRootNonNullToken(rewrittenTree);
       return rewrittenTree;
     } else {
       return ast;

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
index 865c03a..64f1bdd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
@@ -299,13 +299,10 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     }
     rewrittenCtx.setCmd(rewrittenQueryStr.toString());
 
-    ParseDriver pd = new ParseDriver();
     ASTNode rewrittenTree;
     try {
       LOG.info("Going to reparse <" + originalQuery + "> as \n<" + rewrittenQueryStr.toString() + ">");
-      rewrittenTree = pd.parse(rewrittenQueryStr.toString(), rewrittenCtx);
-      rewrittenTree = ParseUtils.findRootNonNullToken(rewrittenTree);
-
+      rewrittenTree = ParseUtils.parse(rewrittenQueryStr.toString(), rewrittenCtx);
     } catch (ParseException e) {
       throw new SemanticException(ErrorMsg.UPDATEDELETE_PARSE_ERROR.getMsg(), e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/java/org/apache/hadoop/hive/ql/tools/LineageInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/tools/LineageInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/tools/LineageInfo.java
index 12154c9..aca8354 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/tools/LineageInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/tools/LineageInfo.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.tools;
 
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.LinkedHashMap;
@@ -109,8 +111,7 @@ public class LineageInfo implements NodeProcessor {
     /*
      * Get the AST tree
      */
-    ParseDriver pd = new ParseDriver();
-    ASTNode tree = pd.parse(query);
+    ASTNode tree = ParseUtils.parse(query, null);
 
     while ((tree.getToken() == null) && (tree.getChildCount() > 0)) {
       tree = (ASTNode) tree.getChild(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
index c659806..c734988 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
@@ -35,7 +35,6 @@ import org.junit.Test;
 
 public class TestMacroSemanticAnalyzer {
 
-  private ParseDriver parseDriver;
   private MacroSemanticAnalyzer analyzer;
   private QueryState queryState;
   private HiveConf conf;
@@ -47,12 +46,11 @@ public class TestMacroSemanticAnalyzer {
     conf = queryState.getConf();
     SessionState.start(conf);
     context = new Context(conf);
-    parseDriver = new ParseDriver();
     analyzer = new MacroSemanticAnalyzer(queryState);
   }
 
   private ASTNode parse(String command) throws Exception {
-    return ParseUtils.findRootNonNullToken(parseDriver.parse(command));
+    return ParseUtils.parse(command);
   }
   private void analyze(ASTNode ast) throws Exception {
     analyzer.analyze(ast, context);

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
index d6fe540..a573808 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
@@ -258,9 +258,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     ctx.setCmd(query);
     ctx.setHDFSCleanup(true);
 
-    ParseDriver pd = new ParseDriver();
-    ASTNode tree = pd.parse(query, ctx);
-    tree = ParseUtils.findRootNonNullToken(tree);
+    ASTNode tree = ParseUtils.parse(query, ctx);
 
     BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(queryState, tree);
     SessionState.get().initTxnMgr(conf);

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java
index e8e29ee..d0395dd 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java
@@ -67,7 +67,7 @@ public class AuthorizationTestUtil {
   }
 
   private static ASTNode parse(String command) throws Exception {
-    return ParseUtils.findRootNonNullToken((new ParseDriver()).parse(command));
+    return ParseUtils.parse(command);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/test/queries/clientpositive/union_pos_alias.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/union_pos_alias.q b/ql/src/test/queries/clientpositive/union_pos_alias.q
new file mode 100644
index 0000000..c4eca68
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/union_pos_alias.q
@@ -0,0 +1,30 @@
+set hive.mapred.mode=nonstrict;
+
+
+explain 
+select 'tst1' as key, count(1) as value from src s1
+UNION ALL
+select key, value from (select 'tst2' as key, count(1) as value from src s2 UNION ALL select 'tst3' as key, count(1) as value from src s3) s4
+order by 1;
+
+select 'tst1' as key, count(1) as value from src s1
+UNION ALL
+select key, value from (select 'tst2' as key, count(1) as value from src s2 UNION ALL select 'tst3' as key, count(1) as value from src s3) s4
+order by 1;
+
+drop table src_10;
+create table src_10 as select * from src limit 10;
+
+explain 
+select key as value, value as key from src_10
+UNION ALL
+select 'test', value from src_10 s3
+order by 2, 1 desc;
+
+
+select key as value, value as key from src_10
+UNION ALL
+select 'test', value from src_10 s3
+order by 2, 1 desc;
+
+drop table src_10;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/test/results/clientpositive/constant_prop_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/constant_prop_1.q.out b/ql/src/test/results/clientpositive/constant_prop_1.q.out
index aaa1dac..3ba1f15 100644
--- a/ql/src/test/results/clientpositive/constant_prop_1.q.out
+++ b/ql/src/test/results/clientpositive/constant_prop_1.q.out
@@ -99,7 +99,7 @@ STAGE PLANS:
               Union
                 Statistics: Num rows: 1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  Statistics: Num rows: 1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 1
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -115,7 +115,7 @@ STAGE PLANS:
               Union
                 Statistics: Num rows: 1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  Statistics: Num rows: 1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1000 Data size: 8000 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 1
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/78e4bb79/ql/src/test/results/clientpositive/union_pos_alias.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_pos_alias.q.out b/ql/src/test/results/clientpositive/union_pos_alias.q.out
new file mode 100644
index 0000000..8eddbd9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/union_pos_alias.q.out
@@ -0,0 +1,308 @@
+PREHOOK: query: explain 
+select 'tst1' as key, count(1) as value from src s1
+UNION ALL
+select key, value from (select 'tst2' as key, count(1) as value from src s2 UNION ALL select 'tst3' as key, count(1) as value from src s3) s4
+order by 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select 'tst1' as key, count(1) as value from src s1
+UNION ALL
+select key, value from (select 'tst2' as key, count(1) as value from src s2 UNION ALL select 'tst3' as key, count(1) as value from src s3) s4
+order by 1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1, Stage-3, Stage-4
+  Stage-3 is a root stage
+  Stage-4 is a root stage
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: s1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              Group By Operator
+                aggregations: count(1)
+                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)
+      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
+          Select Operator
+            expressions: 'tst1' (type: string), _col0 (type: bigint)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+            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
+            Union
+              Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: bigint)
+          TableScan
+            Union
+              Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: bigint)
+          TableScan
+            Union
+              Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: bigint)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 3 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 3 Data size: 288 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-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: s2
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              Group By Operator
+                aggregations: count(1)
+                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)
+      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
+          Select Operator
+            expressions: 'tst2' (type: string), _col0 (type: bigint)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+            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-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: s3
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+              Group By Operator
+                aggregations: count(1)
+                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)
+      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
+          Select Operator
+            expressions: 'tst3' (type: string), _col0 (type: bigint)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
+            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-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select 'tst1' as key, count(1) as value from src s1
+UNION ALL
+select key, value from (select 'tst2' as key, count(1) as value from src s2 UNION ALL select 'tst3' as key, count(1) as value from src s3) s4
+order by 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select 'tst1' as key, count(1) as value from src s1
+UNION ALL
+select key, value from (select 'tst2' as key, count(1) as value from src s2 UNION ALL select 'tst3' as key, count(1) as value from src s3) s4
+order by 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+tst1	500
+tst2	500
+tst3	500
+PREHOOK: query: drop table src_10
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table src_10
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table src_10 as select * from src limit 10
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_10
+POSTHOOK: query: create table src_10 as select * from src limit 10
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_10
+POSTHOOK: Lineage: src_10.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_10.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: explain 
+select key as value, value as key from src_10
+UNION ALL
+select 'test', value from src_10 s3
+order by 2, 1 desc
+PREHOOK: type: QUERY
+POSTHOOK: query: explain 
+select key as value, value as key from src_10
+UNION ALL
+select 'test', value from src_10 s3
+order by 2, 1 desc
+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: src_10
+            Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+              Union
+                Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: string), _col0 (type: string)
+                  sort order: +-
+                  Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+          TableScan
+            alias: s3
+            Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: 'test' (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+              Union
+                Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: string), _col0 (type: string)
+                  sort order: +-
+                  Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 20 Data size: 208 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 key as value, value as key from src_10
+UNION ALL
+select 'test', value from src_10 s3
+order by 2, 1 desc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_10
+#### A masked pattern was here ####
+POSTHOOK: query: select key as value, value as key from src_10
+UNION ALL
+select 'test', value from src_10 s3
+order by 2, 1 desc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_10
+#### A masked pattern was here ####
+test	val_165
+165	val_165
+test	val_238
+238	val_238
+test	val_255
+255	val_255
+test	val_27
+27	val_27
+test	val_278
+278	val_278
+test	val_311
+311	val_311
+test	val_409
+409	val_409
+test	val_484
+484	val_484
+test	val_86
+86	val_86
+test	val_98
+98	val_98
+PREHOOK: query: drop table src_10
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@src_10
+PREHOOK: Output: default@src_10
+POSTHOOK: query: drop table src_10
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@src_10
+POSTHOOK: Output: default@src_10


[36/50] [abbrv] hive git commit: HIVE-15796: HoS: poor reducer parallelism when operator stats are not accurate (Chao Sun, reviewed by Xuefu Zhang)

Posted by se...@apache.org.
HIVE-15796: HoS: poor reducer parallelism when operator stats are not accurate (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/806d6e1b
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/806d6e1b
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/806d6e1b

Branch: refs/heads/hive-14535
Commit: 806d6e1b01640e890fa751017d21fc4b107e4f0a
Parents: 8ab1889
Author: Chao Sun <su...@apache.org>
Authored: Fri Feb 17 12:22:45 2017 -0800
Committer: Chao Sun <su...@apache.org>
Committed: Wed Feb 22 09:28:56 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   7 +-
 .../test/resources/testconfiguration.properties |   3 +-
 .../spark/SetSparkReducerParallelism.java       |  79 ++++-
 .../hive/ql/parse/spark/GenSparkUtils.java      |  24 +-
 .../hive/ql/parse/spark/SparkCompiler.java      |  23 +-
 .../queries/clientpositive/spark_use_op_stats.q |  41 +++
 .../spark/spark_use_op_stats.q.out              | 331 +++++++++++++++++++
 7 files changed, 481 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/806d6e1b/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 3777fa9..0b315e1 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3202,8 +3202,13 @@ public class HiveConf extends Configuration {
             Constants.LLAP_LOGGER_NAME_CONSOLE),
         "logger used for llap-daemons."),
 
+    SPARK_USE_OP_STATS("hive.spark.use.op.stats", true,
+        "Whether to use operator stats to determine reducer parallelism for Hive on Spark. "
+            + "If this is false, Hive will use source table stats to determine reducer "
+            + "parallelism for all first level reduce tasks, and the maximum reducer parallelism "
+            + "from all parents for all the rest (second level and onward) reducer tasks."),
     SPARK_USE_FILE_SIZE_FOR_MAPJOIN("hive.spark.use.file.size.for.mapjoin", false,
-        "If this is set to true, mapjoin optimization in Hive/Spark will use source file sizes associated"
+        "If this is set to true, mapjoin optimization in Hive/Spark will use source file sizes associated "
             + "with TableScan operator on the root of operator tree, instead of using operator statistics."),
     SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout",
       "60s", new TimeValidator(TimeUnit.SECONDS),

http://git-wip-us.apache.org/repos/asf/hive/blob/806d6e1b/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 4a69bcc..d344464 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1483,7 +1483,8 @@ spark.only.query.files=spark_combine_equivalent_work.q,\
   spark_dynamic_partition_pruning.q,\
   spark_dynamic_partition_pruning_2.q,\
   spark_vectorized_dynamic_partition_pruning.q,\
-  spark_use_file_size_for_mapjoin.q
+  spark_use_file_size_for_mapjoin.q,\
+  spark_use_op_stats.q
 
 miniSparkOnYarn.query.files=auto_sortmerge_join_16.q,\
   bucket4.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/806d6e1b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
index 7a5b71f..337f418 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.optimizer.spark;
 
 import java.util.List;
+import java.util.Set;
 import java.util.Stack;
 
 import org.slf4j.Logger;
@@ -29,7 +30,9 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.LimitOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities;
 import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession;
@@ -57,6 +60,12 @@ public class SetSparkReducerParallelism implements NodeProcessor {
 
   // Spark memory per task, and total number of cores
   private ObjectPair<Long, Integer> sparkMemoryAndCores;
+  private final boolean useOpStats;
+
+  public SetSparkReducerParallelism(HiveConf conf) {
+    sparkMemoryAndCores = null;
+    useOpStats = conf.getBoolVar(HiveConf.ConfVars.SPARK_USE_OP_STATS);
+  }
 
   @Override
   public Object process(Node nd, Stack<Node> stack,
@@ -67,16 +76,28 @@ public class SetSparkReducerParallelism implements NodeProcessor {
 
     ReduceSinkOperator sink = (ReduceSinkOperator) nd;
     ReduceSinkDesc desc = sink.getConf();
+    Set<ReduceSinkOperator> parentSinks = null;
 
     int maxReducers = context.getConf().getIntVar(HiveConf.ConfVars.MAXREDUCERS);
     int constantReducers = context.getConf().getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS);
 
+    if (!useOpStats) {
+      parentSinks = OperatorUtils.findOperatorsUpstream(sink, ReduceSinkOperator.class);
+      parentSinks.remove(sink);
+      if (!context.getVisitedReduceSinks().containsAll(parentSinks)) {
+        // We haven't processed all the parent sinks, and we need
+        // them to be done in order to compute the parallelism for this sink.
+        // In this case, skip. We should visit this again from another path.
+        LOG.debug("Skipping sink " + sink + " for now as we haven't seen all its parents.");
+        return false;
+      }
+    }
+
     if (context.getVisitedReduceSinks().contains(sink)) {
       // skip walking the children
       LOG.debug("Already processed reduce sink: " + sink.getName());
       return true;
     }
-
     context.getVisitedReduceSinks().add(sink);
 
     if (needSetParallelism(sink, context.getConf())) {
@@ -96,19 +117,52 @@ public class SetSparkReducerParallelism implements NodeProcessor {
             return false;
           }
         }
+
         long numberOfBytes = 0;
 
-        // we need to add up all the estimates from the siblings of this reduce sink
-        for (Operator<? extends OperatorDesc> sibling
-          : sink.getChildOperators().get(0).getParentOperators()) {
-          if (sibling.getStatistics() != null) {
-            numberOfBytes += sibling.getStatistics().getDataSize();
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Sibling " + sibling + " has stats: " + sibling.getStatistics());
+        if (useOpStats) {
+          // we need to add up all the estimates from the siblings of this reduce sink
+          for (Operator<? extends OperatorDesc> sibling
+              : sink.getChildOperators().get(0).getParentOperators()) {
+            if (sibling.getStatistics() != null) {
+              numberOfBytes += sibling.getStatistics().getDataSize();
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Sibling " + sibling + " has stats: " + sibling.getStatistics());
+              }
+            } else {
+              LOG.warn("No stats available from: " + sibling);
             }
-          } else {
-            LOG.warn("No stats available from: " + sibling);
           }
+        } else if (parentSinks.isEmpty()) {
+          // Not using OP stats and this is the first sink in the path, meaning that
+          // we should use TS stats to infer parallelism
+          for (Operator<? extends OperatorDesc> sibling
+              : sink.getChildOperators().get(0).getParentOperators()) {
+            Set<TableScanOperator> sources =
+                OperatorUtils.findOperatorsUpstream(sibling, TableScanOperator.class);
+            for (TableScanOperator source : sources) {
+              if (source.getStatistics() != null) {
+                numberOfBytes += source.getStatistics().getDataSize();
+                if (LOG.isDebugEnabled()) {
+                  LOG.debug("Table source " + source + " has stats: " + source.getStatistics());
+                }
+              } else {
+                LOG.warn("No stats available from table source: " + source);
+              }
+            }
+          }
+          LOG.debug("Gathered stats for sink " + sink + ". Total size is "
+              + numberOfBytes + " bytes.");
+        } else {
+          // Use the maximum parallelism from all parent reduce sinks
+          int numberOfReducers = 0;
+          for (ReduceSinkOperator parent : parentSinks) {
+            numberOfReducers = Math.max(numberOfReducers, parent.getConf().getNumReducers());
+          }
+          desc.setNumReducers(numberOfReducers);
+          LOG.debug("Set parallelism for sink " + sink + " to " + numberOfReducers
+              + " based on its parents");
+          return false;
         }
 
         // Divide it by 2 so that we can have more reducers
@@ -134,7 +188,7 @@ public class SetSparkReducerParallelism implements NodeProcessor {
         desc.setNumReducers(numReducers);
       }
     } else {
-      LOG.info("Number of reducers determined to be: " + desc.getNumReducers());
+      LOG.info("Number of reducers for sink " + sink + " was already determined to be: " + desc.getNumReducers());
     }
 
     return false;
@@ -165,6 +219,9 @@ public class SetSparkReducerParallelism implements NodeProcessor {
   }
 
   private void getSparkMemoryAndCores(OptimizeSparkProcContext context) throws SemanticException {
+    if (sparkMemoryAndCores != null) {
+      return;
+    }
     if (context.getConf().getBoolean(SPARK_DYNAMIC_ALLOCATION_ENABLED, false)) {
       // If dynamic allocation is enabled, numbers for memory and cores are meaningless. So, we don't
       // try to get it.

http://git-wip-us.apache.org/repos/asf/hive/blob/806d6e1b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
index 36bde30..d0a82af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
@@ -102,21 +102,21 @@ public class GenSparkUtils {
     reduceWork.setReducer(root);
     reduceWork.setNeedsTagging(GenMapRedUtils.needsTagging(reduceWork));
 
-    // All parents should be reduce sinks. We pick the one we just walked
-    // to choose the number of reducers. In the join/union case they will
-    // all be -1. In sort/order case where it matters there will be only
-    // one parent.
-    Preconditions.checkArgument(context.parentOfRoot instanceof ReduceSinkOperator,
-      "AssertionError: expected context.parentOfRoot to be an instance of ReduceSinkOperator, but was "
-      + context.parentOfRoot.getClass().getName());
-    ReduceSinkOperator reduceSink = (ReduceSinkOperator) context.parentOfRoot;
-
-    reduceWork.setNumReduceTasks(reduceSink.getConf().getNumReducers());
+    // Pick the maximum # reducers across all parents as the # of reduce tasks.
+    int maxExecutors = -1;
+    for (Operator<? extends OperatorDesc> parentOfRoot : root.getParentOperators()) {
+      Preconditions.checkArgument(parentOfRoot instanceof ReduceSinkOperator,
+          "AssertionError: expected parentOfRoot to be an "
+              + "instance of ReduceSinkOperator, but was "
+              + parentOfRoot.getClass().getName());
+      ReduceSinkOperator reduceSink = (ReduceSinkOperator) parentOfRoot;
+      maxExecutors = Math.max(maxExecutors, reduceSink.getConf().getNumReducers());
+    }
+    reduceWork.setNumReduceTasks(maxExecutors);
 
+    ReduceSinkOperator reduceSink = (ReduceSinkOperator) context.parentOfRoot;
     setupReduceSink(context, reduceWork, reduceSink);
-
     sparkWork.add(reduceWork);
-
     SparkEdgeProperty edgeProp = getEdgeProperty(reduceSink, reduceWork);
 
     sparkWork.connect(context.preceedingWork, reduceWork, edgeProp);

http://git-wip-us.apache.org/repos/asf/hive/blob/806d6e1b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
index c4b1640..682b987 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hive.ql.lib.GraphWalker;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.lib.PreOrderWalker;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.lib.TypeRule;
@@ -117,6 +118,9 @@ public class SparkCompiler extends TaskCompiler {
     // Annotation OP tree with statistics
     runStatsAnnotation(procCtx);
 
+    // Set reducer parallelism
+    runSetReducerParallelism(procCtx);
+
     // Run Join releated optimizations
     runJoinOptimizations(procCtx);
 
@@ -266,12 +270,27 @@ public class SparkCompiler extends TaskCompiler {
     }
   }
 
-  private void runJoinOptimizations(OptimizeSparkProcContext procCtx) throws SemanticException {
+  private void runSetReducerParallelism(OptimizeSparkProcContext procCtx) throws SemanticException {
     ParseContext pCtx = procCtx.getParseContext();
     Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
     opRules.put(new RuleRegExp("Set parallelism - ReduceSink",
             ReduceSinkOperator.getOperatorName() + "%"),
-        new SetSparkReducerParallelism());
+        new SetSparkReducerParallelism(pCtx.getConf()));
+
+    // The dispatcher fires the processor corresponding to the closest matching
+    // rule and passes the context along
+    Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+    GraphWalker ogw = new PreOrderWalker(disp);
+
+    // Create a list of topop nodes
+    ArrayList<Node> topNodes = new ArrayList<Node>();
+    topNodes.addAll(pCtx.getTopOps().values());
+    ogw.startWalking(topNodes, null);
+  }
+
+  private void runJoinOptimizations(OptimizeSparkProcContext procCtx) throws SemanticException {
+    ParseContext pCtx = procCtx.getParseContext();
+    Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
 
     opRules.put(new TypeRule(JoinOperator.class), new SparkJoinOptimizer(pCtx));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/806d6e1b/ql/src/test/queries/clientpositive/spark_use_op_stats.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/spark_use_op_stats.q b/ql/src/test/queries/clientpositive/spark_use_op_stats.q
new file mode 100644
index 0000000..b559bc0
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/spark_use_op_stats.q
@@ -0,0 +1,41 @@
+set hive.mapred.mode=nonstrict;
+set hive.spark.use.op.stats=false;
+set hive.auto.convert.join=false;
+set hive.exec.reducers.bytes.per.reducer=500;
+
+EXPLAIN
+SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97;
+
+SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97;
+
+CREATE TEMPORARY TABLE tmp AS
+SELECT * FROM src WHERE key > 50 AND key < 200;
+
+EXPLAIN
+WITH a AS (
+  SELECT src1.key, src2.value
+  FROM tmp src1 JOIN tmp src2 ON (src1.key = src2.key)
+  WHERE src1.key > 100
+),
+b AS (
+  SELECT src1.key, src2.value
+  FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+  WHERE src1.key > 150
+)
+SELECT sum(hash(a.key, b.value)) FROM a JOIN b ON a.key = b.key;
+
+WITH a AS (
+  SELECT src1.key, src2.value
+  FROM tmp src1 JOIN tmp src2 ON (src1.key = src2.key)
+  WHERE src1.key > 100
+),
+b AS (
+  SELECT src1.key, src2.value
+  FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+  WHERE src1.key > 150
+)
+SELECT sum(hash(a.key, b.value)) FROM a JOIN b ON a.key = b.key;

http://git-wip-us.apache.org/repos/asf/hive/blob/806d6e1b/ql/src/test/results/clientpositive/spark/spark_use_op_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/spark_use_op_stats.q.out b/ql/src/test/results/clientpositive/spark/spark_use_op_stats.q.out
new file mode 100644
index 0000000..76f9936
--- /dev/null
+++ b/ql/src/test/results/clientpositive/spark/spark_use_op_stats.q.out
@@ -0,0 +1,331 @@
+PREHOOK: query: EXPLAIN
+SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Spark
+      Edges:
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 43), Map 3 (PARTITION-LEVEL SORT, 43)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src1
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToDouble(key) = 97.0) (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: src2
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToDouble(key) = 97.0) (type: boolean)
+                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
+        Reducer 2 
+            Reduce Operator Tree:
+              Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string), _col2 (type: string)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.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 src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT src1.key, src2.value
+FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+WHERE src1.key = 97
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+97	val_97
+97	val_97
+97	val_97
+97	val_97
+PREHOOK: query: CREATE TEMPORARY TABLE tmp AS
+SELECT * FROM src WHERE key > 50 AND key < 200
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tmp
+POSTHOOK: query: CREATE TEMPORARY TABLE tmp AS
+SELECT * FROM src WHERE key > 50 AND key < 200
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tmp
+PREHOOK: query: EXPLAIN
+WITH a AS (
+  SELECT src1.key, src2.value
+  FROM tmp src1 JOIN tmp src2 ON (src1.key = src2.key)
+  WHERE src1.key > 100
+),
+b AS (
+  SELECT src1.key, src2.value
+  FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+  WHERE src1.key > 150
+)
+SELECT sum(hash(a.key, b.value)) FROM a JOIN b ON a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+WITH a AS (
+  SELECT src1.key, src2.value
+  FROM tmp src1 JOIN tmp src2 ON (src1.key = src2.key)
+  WHERE src1.key > 100
+),
+b AS (
+  SELECT src1.key, src2.value
+  FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+  WHERE src1.key > 150
+)
+SELECT sum(hash(a.key, b.value)) FROM a JOIN b ON a.key = b.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Spark
+      Edges:
+        Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 13), Map 5 (PARTITION-LEVEL SORT, 13)
+        Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 43), Reducer 7 (PARTITION-LEVEL SORT, 43)
+        Reducer 4 <- Reducer 3 (GROUP, 1)
+        Reducer 7 <- Map 6 (PARTITION-LEVEL SORT, 43), Map 8 (PARTITION-LEVEL SORT, 43)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src1
+                  Statistics: Num rows: 148 Data size: 1542 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((UDFToDouble(key) > 100.0) and (UDFToDouble(key) > 150.0)) (type: boolean)
+                    Statistics: Num rows: 16 Data size: 166 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 16 Data size: 166 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: 16 Data size: 166 Basic stats: COMPLETE Column stats: NONE
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: src2
+                  Statistics: Num rows: 148 Data size: 1542 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((UDFToDouble(key) > 100.0) and (UDFToDouble(key) > 150.0)) (type: boolean)
+                    Statistics: Num rows: 16 Data size: 166 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 16 Data size: 166 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: 16 Data size: 166 Basic stats: COMPLETE Column stats: NONE
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: src1
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((UDFToDouble(key) > 150.0) and (UDFToDouble(key) > 100.0)) (type: boolean)
+                    Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
+                      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
+        Map 8 
+            Map Operator Tree:
+                TableScan
+                  alias: src2
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((UDFToDouble(key) > 150.0) and (UDFToDouble(key) > 100.0)) (type: boolean)
+                    Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
+                      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)
+        Reducer 2 
+            Reduce Operator Tree:
+              Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 17 Data size: 182 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: 17 Data size: 182 Basic stats: COMPLETE Column stats: NONE
+        Reducer 3 
+            Reduce Operator Tree:
+              Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 66 Data size: 706 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: hash(_col0,_col2) (type: int)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 66 Data size: 706 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    aggregations: sum(_col0)
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: bigint)
+        Reducer 4 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 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
+        Reducer 7 
+            Reduce Operator Tree:
+              Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 60 Data size: 642 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string), _col2 (type: string)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 60 Data size: 642 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: 60 Data size: 642 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: string)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: WITH a AS (
+  SELECT src1.key, src2.value
+  FROM tmp src1 JOIN tmp src2 ON (src1.key = src2.key)
+  WHERE src1.key > 100
+),
+b AS (
+  SELECT src1.key, src2.value
+  FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+  WHERE src1.key > 150
+)
+SELECT sum(hash(a.key, b.value)) FROM a JOIN b ON a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@tmp
+#### A masked pattern was here ####
+POSTHOOK: query: WITH a AS (
+  SELECT src1.key, src2.value
+  FROM tmp src1 JOIN tmp src2 ON (src1.key = src2.key)
+  WHERE src1.key > 100
+),
+b AS (
+  SELECT src1.key, src2.value
+  FROM src src1 JOIN src src2 ON (src1.key = src2.key)
+  WHERE src1.key > 150
+)
+SELECT sum(hash(a.key, b.value)) FROM a JOIN b ON a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@tmp
+#### A masked pattern was here ####
+180817551380


[16/50] [abbrv] hive git commit: HIVE-15970 Fix merge to work in presence of AST rewrites (Eugene Koifman, reviewed by Wei Zheng)

Posted by se...@apache.org.
HIVE-15970 Fix merge to work in presence of AST rewrites (Eugene Koifman, reviewed by Wei Zheng)


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

Branch: refs/heads/hive-14535
Commit: 1a6902ce81c3ac5da98ee5183fa24b98c63642fb
Parents: 3f986d7
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Sat Feb 18 08:58:07 2017 -0800
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Sat Feb 18 08:58:07 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/Context.java | 109 +++++++++++++++----
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |  38 ++++---
 2 files changed, 111 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1a6902ce/ql/src/java/org/apache/hadoop/hive/ql/Context.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
index bbdce63..fae2a12 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
@@ -25,7 +25,6 @@ import java.net.URI;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.HashMap;
-import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -54,6 +53,7 @@ import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.ExplainConfiguration;
 import org.apache.hadoop.hive.ql.parse.ExplainConfiguration.AnalyzeState;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.shims.ShimLoader;
@@ -132,7 +132,17 @@ public class Context {
    * given tree but multi-insert has several and multi-insert representing MERGE must use
    * different prefixes to encode the purpose of different Insert branches
    */
-  private Map<ASTNode, DestClausePrefix> tree2DestNamePrefix;
+  private Map<Integer, DestClausePrefix> insertBranchToNamePrefix = new HashMap<>();
+  private Operation operation = Operation.OTHER;
+  public void setOperation(Operation operation) {
+    this.operation = operation;
+  }
+
+  /**
+   * These ops require special handling in various places
+   * (note that Insert into Acid table is in OTHER category)
+   */
+  public enum Operation {UPDATE, DELETE, MERGE, OTHER};
   public enum DestClausePrefix {
     INSERT("insclause-"), UPDATE("updclause-"), DELETE("delclause-");
     private final String prefix;
@@ -143,36 +153,95 @@ public class Context {
       return prefix;
     }
   }
+  private String getMatchedText(ASTNode n) {
+    return getTokenRewriteStream().toString(n.getTokenStartIndex(), n.getTokenStopIndex() + 1).trim();
+  }
   /**
    * The suffix is always relative to a given ASTNode
    */
   public DestClausePrefix getDestNamePrefix(ASTNode curNode) {
-    //if there is no mapping, we want to default to "old" naming
     assert curNode != null : "must supply curNode";
-    if(tree2DestNamePrefix == null || tree2DestNamePrefix.isEmpty()) {
+    if(curNode.getType() != HiveParser.TOK_INSERT_INTO) {
+      //select statement
+      assert curNode.getType() == HiveParser.TOK_DESTINATION;
+      if(operation == Operation.OTHER) {
+        //not an 'interesting' op
+        return DestClausePrefix.INSERT;
+      }
+      //if it is an 'interesting' op but it's a select it must be a sub-query or a derived table
+      //it doesn't require a special Acid code path - the reset of the code here is to ensure
+      //the tree structure is what we expect
+      boolean thisIsInASubquery = false;
+      parentLoop: while(curNode.getParent() != null) {
+        curNode = (ASTNode) curNode.getParent();
+        switch (curNode.getType()) {
+          case HiveParser.TOK_SUBQUERY_EXPR:
+            //this is a real subquery (foo IN (select ...))
+          case HiveParser.TOK_SUBQUERY:
+            //this is a Derived Table Select * from (select a from ...))
+            //strictly speaking SetOps should have a TOK_SUBQUERY parent so next 6 items are redundant
+          case HiveParser.TOK_UNIONALL:
+          case HiveParser.TOK_UNIONDISTINCT:
+          case HiveParser.TOK_EXCEPTALL:
+          case HiveParser.TOK_EXCEPTDISTINCT:
+          case HiveParser.TOK_INTERSECTALL:
+          case HiveParser.TOK_INTERSECTDISTINCT:
+            thisIsInASubquery = true;
+            break parentLoop;
+        }
+      }
+      if(!thisIsInASubquery) {
+        throw new IllegalStateException("Expected '" + getMatchedText(curNode) + "' to be in sub-query or set operation.");
+      } 
       return DestClausePrefix.INSERT;
     }
-    do {
-      DestClausePrefix prefix = tree2DestNamePrefix.get(curNode);
-      if(prefix != null) {
-        return prefix;
-      }
-      curNode = (ASTNode) curNode.parent;
-    } while(curNode != null);
-    return DestClausePrefix.INSERT;
+    switch (operation) {
+      case OTHER:
+        return DestClausePrefix.INSERT;
+      case UPDATE:
+        return DestClausePrefix.UPDATE;
+      case DELETE:
+        return DestClausePrefix.DELETE;
+      case MERGE:
+      /* This is the structrue expected here
+        HiveParser.TOK_QUERY;
+          HiveParser.TOK_FROM
+          HiveParser.TOK_INSERT;
+            HiveParser.TOK_INSERT_INTO;
+          HiveParser.TOK_INSERT;
+            HiveParser.TOK_INSERT_INTO;
+          .....*/
+        ASTNode insert = (ASTNode) curNode.getParent();
+        assert insert != null && insert.getType() == HiveParser.TOK_INSERT;
+        ASTNode query = (ASTNode) insert.getParent();
+        assert query != null && query.getType() == HiveParser.TOK_QUERY;
+        
+        for(int childIdx = 1; childIdx < query.getChildCount(); childIdx++) {//1st child is TOK_FROM
+          assert query.getChild(childIdx).getType() == HiveParser.TOK_INSERT;
+          if(insert == query.getChild(childIdx)) {
+            DestClausePrefix prefix = insertBranchToNamePrefix.get(childIdx);
+            if(prefix == null) {
+              throw new IllegalStateException("Found a node w/o branch mapping: '" +
+                getMatchedText(insert) + "'");
+            }
+            return prefix;
+          }
+        }
+        throw new IllegalStateException("Could not locate '" + getMatchedText(insert) + "'");
+      default:
+        throw new IllegalStateException("Unexpected operation: " + operation);
+    }
   }
   /**
-   * Will make SemanticAnalyzer.Phase1Ctx#dest in subtree rooted at 'tree' use 'prefix'
-   * @param tree
+   * Will make SemanticAnalyzer.Phase1Ctx#dest in subtree rooted at 'tree' use 'prefix'.  This to
+   * handle multi-insert stmt that represents Merge stmt and has insert branches representing
+   * update/delete/insert.
+   * @param pos ordinal index of specific TOK_INSERT as child of TOK_QUERY
    * @return previous prefix for 'tree' or null
    */
-  public DestClausePrefix addDestNamePrefix(ASTNode tree, DestClausePrefix prefix) {
-    if(tree2DestNamePrefix == null) {
-      tree2DestNamePrefix = new IdentityHashMap<>();
-    }
-    return tree2DestNamePrefix.put(tree, prefix);
+  public DestClausePrefix addDestNamePrefix(int pos, DestClausePrefix prefix) {
+    return insertBranchToNamePrefix.put(pos, prefix);
   }
-
   public Context(Configuration conf) throws IOException {
     this(conf, generateExecutionId());
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/1a6902ce/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
index 12a43a2..725f2ce 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
@@ -58,9 +58,9 @@ import org.apache.hadoop.hive.ql.session.SessionState;
  */
 public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
 
-  boolean useSuper = false;
+  private boolean useSuper = false;
 
-  public UpdateDeleteSemanticAnalyzer(QueryState queryState) throws SemanticException {
+  UpdateDeleteSemanticAnalyzer(QueryState queryState) throws SemanticException {
     super(queryState);
   }
 
@@ -92,19 +92,19 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     }
   }
   private boolean updating() {
-    return currentOperation == Operation.UPDATE;
+    return currentOperation == Context.Operation.UPDATE;
   }
   private boolean deleting() {
-    return currentOperation == Operation.DELETE;
+    return currentOperation == Context.Operation.DELETE;
   }
 
   private void analyzeUpdate(ASTNode tree) throws SemanticException {
-    currentOperation = Operation.UPDATE;
+    currentOperation = Context.Operation.UPDATE;
     reparseAndSuperAnalyze(tree);
   }
 
   private void analyzeDelete(ASTNode tree) throws SemanticException {
-    currentOperation = Operation.DELETE;
+    currentOperation = Context.Operation.DELETE;
     reparseAndSuperAnalyze(tree);
   }
   /**
@@ -410,10 +410,12 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
         "Expected TOK_INSERT as second child of TOK_QUERY but found " + rewrittenInsert.getName();
 
     if(updating()) {
-      rewrittenCtx.addDestNamePrefix(rewrittenInsert, Context.DestClausePrefix.UPDATE);
+      rewrittenCtx.setOperation(Context.Operation.UPDATE);
+      rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.UPDATE);
     }
     else if(deleting()) {
-      rewrittenCtx.addDestNamePrefix(rewrittenInsert, Context.DestClausePrefix.DELETE);
+      rewrittenCtx.setOperation(Context.Operation.DELETE);
+      rewrittenCtx.addDestNamePrefix(1, Context.DestClausePrefix.DELETE);
     }
 
     if (where != null) {
@@ -489,7 +491,7 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     return false;
   }
   private String operation() {
-    if (currentOperation == Operation.NOT_ACID) {
+    if (currentOperation == Context.Operation.OTHER) {
       throw new IllegalStateException("UpdateDeleteSemanticAnalyzer neither updating nor " +
         "deleting, operation not known.");
     }
@@ -523,8 +525,7 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     return colName.toLowerCase();
   }
 
-  private enum Operation {UPDATE, DELETE, MERGE, NOT_ACID};
-  private Operation currentOperation = Operation.NOT_ACID;
+  private Context.Operation currentOperation = Context.Operation.OTHER;
   private static final String Indent = "  ";
 
   private IdentifierQuoter quotedIdenfierHelper;
@@ -589,7 +590,7 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
    * @throws SemanticException
    */
   private void analyzeMerge(ASTNode tree) throws SemanticException {
-    currentOperation = Operation.MERGE;
+    currentOperation = Context.Operation.MERGE;
     quotedIdenfierHelper = new IdentifierQuoter(ctx.getTokenRewriteStream());
     /*
      * See org.apache.hadoop.hive.ql.parse.TestMergeStatement for some examples of the merge AST
@@ -701,8 +702,9 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     ReparseResult rr = parseRewrittenQuery(rewrittenQueryStr, ctx.getCmd());
     Context rewrittenCtx = rr.rewrittenCtx;
     ASTNode rewrittenTree = rr.rewrittenTree;
+    rewrittenCtx.setOperation(Context.Operation.MERGE);
 
-    //set dest name mapping on new context
+    //set dest name mapping on new context; 1st chid is TOK_FROM
     for(int insClauseIdx = 1, whenClauseIdx = 0;
         insClauseIdx < rewrittenTree.getChildCount() - (validating ? 1 : 0/*skip cardinality violation clause*/);
         insClauseIdx++, whenClauseIdx++) {
@@ -710,18 +712,22 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
       ASTNode insertClause = (ASTNode) rewrittenTree.getChild(insClauseIdx);
       switch (getWhenClauseOperation(whenClauses.get(whenClauseIdx)).getType()) {
         case HiveParser.TOK_INSERT:
-          rewrittenCtx.addDestNamePrefix(insertClause, Context.DestClausePrefix.INSERT);
+          rewrittenCtx.addDestNamePrefix(insClauseIdx, Context.DestClausePrefix.INSERT);
           break;
         case HiveParser.TOK_UPDATE:
-          rewrittenCtx.addDestNamePrefix(insertClause, Context.DestClausePrefix.UPDATE);
+          rewrittenCtx.addDestNamePrefix(insClauseIdx, Context.DestClausePrefix.UPDATE);
           break;
         case HiveParser.TOK_DELETE:
-          rewrittenCtx.addDestNamePrefix(insertClause, Context.DestClausePrefix.DELETE);
+          rewrittenCtx.addDestNamePrefix(insClauseIdx, Context.DestClausePrefix.DELETE);
           break;
         default:
           assert false;
       }
     }
+    if(validating) {
+      //here means the last branch of the multi-insert is Cardinality Validation
+      rewrittenCtx.addDestNamePrefix(rewrittenTree.getChildCount() - 1, Context.DestClausePrefix.INSERT);
+    }
     try {
       useSuper = true;
       super.analyze(rewrittenTree, rewrittenCtx);


[15/50] [abbrv] hive git commit: HIVE-15919: Row count mismatch for count * query (Jason Dere, reviewed by Matt McCline)

Posted by se...@apache.org.
HIVE-15919: Row count mismatch for count * query (Jason Dere, reviewed by Matt McCline)


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

Branch: refs/heads/hive-14535
Commit: 3f986d7a286d7777aebce92bb6c62d00795d0e8f
Parents: e618bd1
Author: Jason Dere <jd...@hortonworks.com>
Authored: Fri Feb 17 14:30:30 2017 -0800
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Fri Feb 17 14:30:30 2017 -0800

----------------------------------------------------------------------
 .../ql/exec/vector/VectorizationContext.java    | 112 ++++++++++++++-----
 .../exec/vector/TestVectorizationContext.java   |  37 ++++++
 .../llap/vector_decimal_mapjoin.q.out           |   4 +-
 .../llap/vector_reduce_groupby_decimal.q.out    |   2 +-
 .../spark/vector_decimal_mapjoin.q.out          |   4 +-
 .../clientpositive/vector_decimal_mapjoin.q.out |   2 +-
 .../vector_reduce_groupby_decimal.q.out         |   2 +-
 7 files changed, 125 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3f986d7a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 8164684..bf78251 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -104,20 +104,7 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeDynamicValueDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.GroupByDesc;
 import org.apache.hadoop.hive.ql.udf.SettableUDF;
-import org.apache.hadoop.hive.ql.udf.UDFConv;
-import org.apache.hadoop.hive.ql.udf.UDFFromUnixTime;
-import org.apache.hadoop.hive.ql.udf.UDFHex;
-import org.apache.hadoop.hive.ql.udf.UDFRegExpExtract;
-import org.apache.hadoop.hive.ql.udf.UDFRegExpReplace;
-import org.apache.hadoop.hive.ql.udf.UDFSign;
-import org.apache.hadoop.hive.ql.udf.UDFToBoolean;
-import org.apache.hadoop.hive.ql.udf.UDFToByte;
-import org.apache.hadoop.hive.ql.udf.UDFToDouble;
-import org.apache.hadoop.hive.ql.udf.UDFToFloat;
-import org.apache.hadoop.hive.ql.udf.UDFToInteger;
-import org.apache.hadoop.hive.ql.udf.UDFToLong;
-import org.apache.hadoop.hive.ql.udf.UDFToShort;
-import org.apache.hadoop.hive.ql.udf.UDFToString;
+import org.apache.hadoop.hive.ql.udf.*;
 import org.apache.hadoop.hive.ql.udf.generic.*;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
@@ -359,6 +346,67 @@ public class VectorizationContext {
     castExpressionUdfs.add(UDFToShort.class);
   }
 
+  // Set of GenericUDFs which require need implicit type casting of decimal parameters.
+  // Vectorization for mathmatical functions currently depends on decimal params automatically
+  // being converted to the return type (see getImplicitCastExpression()), which is not correct
+  // in the general case. This set restricts automatic type conversion to just these functions.
+  private static Set<Class<?>> udfsNeedingImplicitDecimalCast = new HashSet<Class<?>>();
+  static {
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPPlus.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPMinus.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPMultiply.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPDivide.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPMod.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFRound.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFBRound.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFFloor.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFCbrt.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFCeil.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFAbs.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFPosMod.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFPower.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFFactorial.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPPositive.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPNegative.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFCoalesce.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFElt.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFGreatest.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFLeast.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFIn.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPEqual.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPEqualNS.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPNotEqual.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPLessThan.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPEqualOrLessThan.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPGreaterThan.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPEqualOrGreaterThan.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFBetween.class);
+    udfsNeedingImplicitDecimalCast.add(UDFSqrt.class);
+    udfsNeedingImplicitDecimalCast.add(UDFRand.class);
+    udfsNeedingImplicitDecimalCast.add(UDFLn.class);
+    udfsNeedingImplicitDecimalCast.add(UDFLog2.class);
+    udfsNeedingImplicitDecimalCast.add(UDFSin.class);
+    udfsNeedingImplicitDecimalCast.add(UDFAsin.class);
+    udfsNeedingImplicitDecimalCast.add(UDFCos.class);
+    udfsNeedingImplicitDecimalCast.add(UDFAcos.class);
+    udfsNeedingImplicitDecimalCast.add(UDFLog10.class);
+    udfsNeedingImplicitDecimalCast.add(UDFLog.class);
+    udfsNeedingImplicitDecimalCast.add(UDFExp.class);
+    udfsNeedingImplicitDecimalCast.add(UDFDegrees.class);
+    udfsNeedingImplicitDecimalCast.add(UDFRadians.class);
+    udfsNeedingImplicitDecimalCast.add(UDFAtan.class);
+    udfsNeedingImplicitDecimalCast.add(UDFTan.class);
+    udfsNeedingImplicitDecimalCast.add(UDFOPLongDivide.class);
+  }
+
+  protected boolean needsImplicitCastForDecimal(GenericUDF udf) {
+    Class<?> udfClass = udf.getClass();
+    if (udf instanceof GenericUDFBridge) {
+      udfClass = ((GenericUDFBridge) udf).getUdfClass();
+    }
+    return udfsNeedingImplicitDecimalCast.contains(udfClass);
+  }
+
   protected int getInputColumnIndex(String name) throws HiveException {
     if (name == null) {
       throw new HiveException("Null column name");
@@ -764,24 +812,26 @@ public class VectorizationContext {
     }
 
     if (castTypeDecimal && !inputTypeDecimal) {
-
-      // Cast the input to decimal
-      // If castType is decimal, try not to lose precision for numeric types.
-      castType = updatePrecision(inputTypeInfo, (DecimalTypeInfo) castType);
-      GenericUDFToDecimal castToDecimalUDF = new GenericUDFToDecimal();
-      castToDecimalUDF.setTypeInfo(castType);
-      List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
-      children.add(child);
-      ExprNodeDesc desc = new ExprNodeGenericFuncDesc(castType, castToDecimalUDF, children);
-      return desc;
+      if (needsImplicitCastForDecimal(udf)) {
+        // Cast the input to decimal
+        // If castType is decimal, try not to lose precision for numeric types.
+        castType = updatePrecision(inputTypeInfo, (DecimalTypeInfo) castType);
+        GenericUDFToDecimal castToDecimalUDF = new GenericUDFToDecimal();
+        castToDecimalUDF.setTypeInfo(castType);
+        List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+        children.add(child);
+        ExprNodeDesc desc = new ExprNodeGenericFuncDesc(castType, castToDecimalUDF, children);
+        return desc;
+      }
     } else if (!castTypeDecimal && inputTypeDecimal) {
-
-      // Cast decimal input to returnType
-      GenericUDF genericUdf = getGenericUDFForCast(castType);
-      List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
-      children.add(child);
-      ExprNodeDesc desc = new ExprNodeGenericFuncDesc(castType, genericUdf, children);
-      return desc;
+      if (needsImplicitCastForDecimal(udf)) {
+        // Cast decimal input to returnType
+        GenericUDF genericUdf = getGenericUDFForCast(castType);
+        List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+        children.add(child);
+        ExprNodeDesc desc = new ExprNodeGenericFuncDesc(castType, genericUdf, children);
+        return desc;
+      }
     } else {
 
       // Casts to exact types including long to double etc. are needed in some special cases.

http://git-wip-us.apache.org/repos/asf/hive/blob/3f986d7a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
index bb37a04..9fcb392 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
@@ -31,11 +31,13 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.BRoundWithNumDigitsDoub
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ColAndCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ColOrCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.DoubleColumnInList;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.DynamicValueVectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterExprAndExpr;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterExprOrExpr;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FuncLogWithBaseDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FuncLogWithBaseLongToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FuncPowerDoubleToDouble;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IdentityExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprCharScalarStringGroupColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprDoubleColumnDoubleColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprLongColumnLongColumn;
@@ -66,6 +68,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringLTrim;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringLower;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringUpper;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorInBloomFilterColDynamicValue;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFUnixTimeStampDate;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFUnixTimeStampTimestamp;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterStringColumnInList;
@@ -110,9 +113,11 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColSubtractLong
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColUnaryMinus;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongScalarSubtractLongColumn;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.DynamicValue;
 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.ExprNodeDynamicValueDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.udf.UDFLog;
 import org.apache.hadoop.hive.ql.udf.UDFSin;
@@ -123,6 +128,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIf;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFInBloomFilter;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLTrim;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLower;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
@@ -1584,4 +1590,35 @@ public class TestVectorizationContext {
     b = 1;
     assertEquals(a != b ? 1 : 0, ((a - b) ^ (b - a)) >>> 63);
   }
+
+  @Test
+  public void testInBloomFilter() throws Exception {
+    // Setup InBloomFilter() UDF
+    ExprNodeColumnDesc colExpr = new ExprNodeColumnDesc(TypeInfoFactory.getDecimalTypeInfo(10, 5), "a", "table", false);
+    ExprNodeDesc bfExpr = new ExprNodeDynamicValueDesc(new DynamicValue("id1", TypeInfoFactory.binaryTypeInfo));
+
+    ExprNodeGenericFuncDesc inBloomFilterExpr = new ExprNodeGenericFuncDesc();
+    GenericUDF inBloomFilterUdf = new GenericUDFInBloomFilter();
+    inBloomFilterExpr.setTypeInfo(TypeInfoFactory.booleanTypeInfo);
+    inBloomFilterExpr.setGenericUDF(inBloomFilterUdf);
+    List<ExprNodeDesc> children1 = new ArrayList<ExprNodeDesc>(2);
+    children1.add(colExpr);
+    children1.add(bfExpr);
+    inBloomFilterExpr.setChildren(children1);
+
+    // Setup VectorizationContext
+    List<String> columns = new ArrayList<String>();
+    columns.add("b");
+    columns.add("a");
+    VectorizationContext vc = new VectorizationContext("name", columns);
+
+    // Create vectorized expr
+    VectorExpression ve = vc.getVectorExpression(inBloomFilterExpr, VectorExpressionDescriptor.Mode.FILTER);
+    Assert.assertEquals(VectorInBloomFilterColDynamicValue.class, ve.getClass());
+    VectorInBloomFilterColDynamicValue vectorizedInBloomFilterExpr = (VectorInBloomFilterColDynamicValue) ve;
+    VectorExpression[] children = vectorizedInBloomFilterExpr.getChildExpressions();
+    // VectorInBloomFilterColDynamicValue should have all of the necessary information to vectorize.
+    // Should be no need for child vector expressions, which would imply casting/conversion.
+    Assert.assertNull(children);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/3f986d7a/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
index 29e779d..6275c59 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
@@ -106,7 +106,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 1)(children: CastDecimalToBoolean(col 0) -> 1:Boolean) -> boolean
+                        predicateExpression: SelectColumnIsNotNull(col 0) -> boolean
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -165,7 +165,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 1)(children: CastDecimalToBoolean(col 0) -> 1:Boolean) -> boolean
+                        predicateExpression: SelectColumnIsNotNull(col 0) -> boolean
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: NONE
                     Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/3f986d7a/ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out b/ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out
index e17aff5..a28719f 100644
--- a/ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_reduce_groupby_decimal.q.out
@@ -57,7 +57,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 4)(children: CastDecimalToBoolean(col 2) -> 4:Boolean) -> boolean, SelectColumnIsNotNull(col 4)(children: CastDecimalToBoolean(col 3) -> 4:Boolean) -> boolean) -> boolean
+                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 2) -> boolean, SelectColumnIsNotNull(col 3) -> boolean) -> boolean
                     predicate: (cdecimal1 is not null and cdecimal2 is not null) (type: boolean)
                     Statistics: Num rows: 6102 Data size: 1440072 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/3f986d7a/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out b/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out
index e4c92c7..968e080 100644
--- a/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out
@@ -104,7 +104,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 1)(children: CastDecimalToBoolean(col 0) -> 1:Boolean) -> boolean
+                        predicateExpression: SelectColumnIsNotNull(col 0) -> boolean
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -150,7 +150,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 1)(children: CastDecimalToBoolean(col 0) -> 1:Boolean) -> boolean
+                        predicateExpression: SelectColumnIsNotNull(col 0) -> boolean
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: NONE
                     Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/3f986d7a/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out b/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out
index c3c0c4e..223a7a8 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out
@@ -124,7 +124,7 @@ STAGE PLANS:
               Filter Vectorization:
                   className: VectorFilterOperator
                   native: true
-                  predicateExpression: SelectColumnIsNotNull(col 1)(children: CastDecimalToBoolean(col 0) -> 1:Boolean) -> boolean
+                  predicateExpression: SelectColumnIsNotNull(col 0) -> boolean
               predicate: dec is not null (type: boolean)
               Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: NONE
               Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/3f986d7a/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out b/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out
index 7094c9c..a6e0c1d 100644
--- a/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out
+++ b/ql/src/test/results/clientpositive/vector_reduce_groupby_decimal.q.out
@@ -51,7 +51,7 @@ STAGE PLANS:
               Filter Vectorization:
                   className: VectorFilterOperator
                   native: true
-                  predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 4)(children: CastDecimalToBoolean(col 2) -> 4:Boolean) -> boolean, SelectColumnIsNotNull(col 4)(children: CastDecimalToBoolean(col 3) -> 4:Boolean) -> boolean) -> boolean
+                  predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 2) -> boolean, SelectColumnIsNotNull(col 3) -> boolean) -> boolean
               predicate: (cdecimal1 is not null and cdecimal2 is not null) (type: boolean)
               Statistics: Num rows: 6102 Data size: 1440072 Basic stats: COMPLETE Column stats: NONE
               Group By Operator


[31/50] [abbrv] hive git commit: HIVE-15934 : Downgrade Maven surefire plugin from 2.19.1 to 2.18.1 (Wei Zheng, reviewed by Zoltan Haindrich)

Posted by se...@apache.org.
HIVE-15934 : Downgrade Maven surefire plugin from 2.19.1 to 2.18.1 (Wei Zheng, reviewed by Zoltan Haindrich)


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

Branch: refs/heads/hive-14535
Commit: ffe735766efa7ac6469aec0c43cc6fcdfab5662c
Parents: d5bb76c
Author: Wei Zheng <we...@apache.org>
Authored: Tue Feb 21 15:46:18 2017 -0800
Committer: Wei Zheng <we...@apache.org>
Committed: Tue Feb 21 15:46:18 2017 -0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ffe73576/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3c46ad1..3ddec7a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -99,7 +99,7 @@
     <maven.jar.plugin.version>2.4</maven.jar.plugin.version>
     <maven.javadoc.plugin.version>2.4</maven.javadoc.plugin.version>
     <maven.shade.plugin.version>2.4.3</maven.shade.plugin.version>
-    <maven.surefire.plugin.version>2.19.1</maven.surefire.plugin.version>
+    <maven.surefire.plugin.version>2.18.1</maven.surefire.plugin.version>
     <maven.war.plugin.version>2.4</maven.war.plugin.version>
     <maven.dependency.plugin.version>2.8</maven.dependency.plugin.version>
     <maven.eclipse.plugin.version>2.9</maven.eclipse.plugin.version>


[48/50] [abbrv] hive git commit: HIVE-16017 : MM tables - many queries duplicate the data after master merge (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-16017 : MM tables - many queries duplicate the data after master merge (Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 1f0a5ef31c450db497052300a44a1080add47c6a
Parents: 74d9333
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Feb 23 16:54:06 2017 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Feb 23 16:54:06 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/Utilities.java   |  2 +-
 ql/src/test/results/clientpositive/mm_all.q.out | 82 ++++++--------------
 2 files changed, 26 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1f0a5ef3/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 80ca28d..d7db991 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
@@ -3187,7 +3187,7 @@ public final class Utilities {
             continue;
           }
 
-          pathsProcessed.add(path);
+          pathsProcessed.add(file);
           if (LOG.isDebugEnabled()) {
             LOG.debug("Adding input file " + file);
           } else if (!hasLogged) {

http://git-wip-us.apache.org/repos/asf/hive/blob/1f0a5ef3/ql/src/test/results/clientpositive/mm_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/mm_all.q.out b/ql/src/test/results/clientpositive/mm_all.q.out
index dd0a3b1..4944d6c 100644
--- a/ql/src/test/results/clientpositive/mm_all.q.out
+++ b/ql/src/test/results/clientpositive/mm_all.q.out
@@ -1,8 +1,6 @@
-PREHOOK: query: -- Force multiple writers when reading
-drop table intermediate
+PREHOOK: query: drop table intermediate
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: -- Force multiple writers when reading
-drop table intermediate
+POSTHOOK: query: drop table intermediate
 POSTHOOK: type: DROPTABLE
 PREHOOK: query: create table intermediate(key int) partitioned by (p int) stored as orc
 PREHOOK: type: CREATETABLE
@@ -205,15 +203,13 @@ POSTHOOK: Input: default@part_mm@key_mm=456
 103	455
 103	455
 103	456
-PREHOOK: query: -- TODO: doesn't work truncate table part_mm partition(key_mm=455);
-select * from part_mm order by key, key_mm
+PREHOOK: query: 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: -- TODO: doesn't work truncate table part_mm partition(key_mm=455);
-select * from part_mm order by key, key_mm
+POSTHOOK: query: 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
@@ -378,11 +374,9 @@ POSTHOOK: query: drop table simple_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@simple_mm
 POSTHOOK: Output: default@simple_mm
-PREHOOK: query: -- simple DP (no bucketing)
-drop table dp_mm
+PREHOOK: query: drop table dp_mm
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: -- simple DP (no bucketing)
-drop table dp_mm
+POSTHOOK: query: drop table dp_mm
 POSTHOOK: type: DROPTABLE
 PREHOOK: query: create table dp_mm (key int) partitioned by (key1 string, key2 int) stored as orc
   tblproperties ("transactional"="true", "transactional_properties"="insert_only")
@@ -453,15 +447,11 @@ POSTHOOK: query: drop table dp_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@dp_mm
 POSTHOOK: Output: default@dp_mm
-PREHOOK: query: -- union
-
-create table union_mm(id int)  tblproperties ("transactional"="true", "transactional_properties"="insert_only")
+PREHOOK: query: create table union_mm(id int)  tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
 PREHOOK: Output: default@union_mm
-POSTHOOK: query: -- union
-
-create table union_mm(id int)  tblproperties ("transactional"="true", "transactional_properties"="insert_only")
+POSTHOOK: query: create table union_mm(id int)  tblproperties ("transactional"="true", "transactional_properties"="insert_only")
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@union_mm
@@ -949,8 +939,8 @@ POSTHOOK: Input: default@skew_dp_union_mm@k3=98
 97	97	97	97
 98	98	98	98
 98	99	100	101
-100	100	100	100
 99	100	101	102
+100	100	100	100
 101	102	103	104
 103	103	103	103
 104	105	106	107
@@ -1175,9 +1165,9 @@ POSTHOOK: Input: default@merge1_mm@key=103
 POSTHOOK: Input: default@merge1_mm@key=97
 POSTHOOK: Input: default@merge1_mm@key=98
 #### A masked pattern was here ####
-98	98
 97	97
 103	103
+98	98
 100	100
 10	10
 0	0
@@ -1227,17 +1217,17 @@ POSTHOOK: Input: default@merge1_mm@key=97
 POSTHOOK: Input: default@merge1_mm@key=98
 #### A masked pattern was here ####
 100	100
-100	100
 97	97
 103	103
 103	103
+100	100
 97	97
 98	98
 98	98
-10	10
-10	10
 0	0
+10	10
 0	0
+10	10
 PREHOOK: query: drop table merge1_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@merge1_mm
@@ -1246,15 +1236,9 @@ POSTHOOK: query: drop table merge1_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@merge1_mm
 POSTHOOK: Output: default@merge1_mm
-PREHOOK: query: -- TODO: need to include merge+union+DP, but it's broken for now
-
-
-drop table ctas0_mm
+PREHOOK: query: drop table ctas0_mm
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: -- TODO: need to include merge+union+DP, but it's broken for now
-
-
-drop table ctas0_mm
+POSTHOOK: query: drop table ctas0_mm
 POSTHOOK: type: DROPTABLE
 PREHOOK: query: create table ctas0_mm tblproperties ("transactional"="true", "transactional_properties"="insert_only") as select * from intermediate
 PREHOOK: type: CREATETABLE_AS_SELECT
@@ -2162,13 +2146,9 @@ POSTHOOK: query: drop table intermmediate_nonpart
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@intermmediate_nonpart
 POSTHOOK: Output: default@intermmediate_nonpart
-PREHOOK: query: -- non-MM export to MM table, with and without partitions
-
-drop table import0_mm
+PREHOOK: query: drop table import0_mm
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: -- non-MM export to MM table, with and without partitions
-
-drop table import0_mm
+POSTHOOK: query: drop table import0_mm
 POSTHOOK: type: DROPTABLE
 PREHOOK: query: create table import0_mm(key int, p int) tblproperties("transactional"="true", "transactional_properties"="insert_only")
 PREHOOK: type: CREATETABLE
@@ -2261,13 +2241,9 @@ POSTHOOK: query: drop table import1_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@import1_mm
 POSTHOOK: Output: default@import1_mm
-PREHOOK: query: -- MM export into new MM table, non-part and part
-
-drop table import2_mm
+PREHOOK: query: drop table import2_mm
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: -- MM export into new MM table, non-part and part
-
-drop table import2_mm
+POSTHOOK: query: drop table import2_mm
 POSTHOOK: type: DROPTABLE
 PREHOOK: query: import table import2_mm from 'ql/test/data/exports/intermmediate_nonpart'
 PREHOOK: type: IMPORT
@@ -2365,13 +2341,9 @@ POSTHOOK: query: drop table import3_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@import3_mm
 POSTHOOK: Output: default@import3_mm
-PREHOOK: query: -- MM export into existing MM table, non-part and partial part
-
-drop table import4_mm
+PREHOOK: query: drop table import4_mm
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: -- MM export into existing MM table, non-part and partial part
-
-drop table import4_mm
+POSTHOOK: query: drop table import4_mm
 POSTHOOK: type: DROPTABLE
 PREHOOK: query: create table import4_mm(key int, p int) tblproperties("transactional"="true", "transactional_properties"="insert_only")
 PREHOOK: type: CREATETABLE
@@ -2452,13 +2424,9 @@ POSTHOOK: query: drop table import5_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@import5_mm
 POSTHOOK: Output: default@import5_mm
-PREHOOK: query: -- MM export into existing non-MM table, non-part and part
-
-drop table import6_mm
+PREHOOK: query: drop table import6_mm
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: -- MM export into existing non-MM table, non-part and part
-
-drop table import6_mm
+POSTHOOK: query: drop table import6_mm
 POSTHOOK: type: DROPTABLE
 PREHOOK: query: create table import6_mm(key int, p int)
 PREHOOK: type: CREATETABLE
@@ -3136,7 +3104,7 @@ 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), ]
+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
@@ -3145,7 +3113,7 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@stats2_mm
 # col_name            	data_type           	comment             
 	 	 
-c0                  	array<string>       	                    
+_c0                 	array<string>       	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 


[14/50] [abbrv] hive git commit: HIVE-15969 : Failures in TestRemoteHiveMetaStore, TestSetUGIOnOnlyServer (Slim Bouguerra via Thejas Nair)

Posted by se...@apache.org.
HIVE-15969 : Failures in TestRemoteHiveMetaStore, TestSetUGIOnOnlyServer (Slim Bouguerra via Thejas Nair)

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


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

Branch: refs/heads/hive-14535
Commit: e618bd1b45bc45f96b27698fd0c97fee6b9c57e6
Parents: 5c29371
Author: Slim Bouguerra <sl...@gmail.com>
Authored: Fri Feb 17 14:15:15 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Feb 17 14:15:15 2017 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/metastore/HiveMetaStoreClient.java    | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e618bd1b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 70f3a6b..7002620 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -739,12 +739,14 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
         hook.commitCreateTable(tbl);
       }
       success = true;
-    } catch (Exception e){
-      LOG.error("Got exception from createTable", e);
     }
     finally {
       if (!success && (hook != null)) {
-        hook.rollbackCreateTable(tbl);
+        try {
+          hook.rollbackCreateTable(tbl);
+        } catch (Exception e){
+          LOG.error("Create rollback failed with", e);
+        }
       }
     }
   }


[50/50] [abbrv] 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:
	metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
	metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
	metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
	metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
	ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java


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

Branch: refs/heads/hive-14535
Commit: 2014ece97960f8d2f690f55c131d9a61c421c2f9
Parents: 1f0a5ef b8d7192
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Feb 23 17:08:03 2017 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Feb 23 17:08:03 2017 -0800

----------------------------------------------------------------------
 ant/pom.xml                                     |   69 -
 .../hive/ant/DistinctElementsClassPath.java     |   90 -
 .../apache/hadoop/hive/ant/GenVectorCode.java   | 3309 --------------
 .../hadoop/hive/ant/GenVectorTestCode.java      |  261 --
 .../apache/hadoop/hive/ant/GetVersionPref.java  |   94 -
 ant/src/org/apache/hadoop/hive/ant/antlib.xml   |   24 -
 .../java/org/apache/hive/beeline/BeeLine.java   |   42 +-
 .../hive/beeline/BeeLineSignalHandler.java      |    1 -
 .../java/org/apache/hive/beeline/Commands.java  |    2 +-
 .../apache/hive/beeline/SunSignalHandler.java   |   13 +-
 beeline/src/main/resources/BeeLine.properties   |    1 +
 .../apache/hive/beeline/TestHiveSchemaTool.java |   73 +
 .../apache/hive/beeline/TestShutdownHook.java   |   46 +
 bin/beeline                                     |    5 -
 bin/beeline.cmd                                 |   66 -
 bin/derbyserver.cmd                             |   60 -
 bin/ext/cleardanglingscratchdir.cmd             |   34 -
 bin/ext/cli.cmd                                 |   58 -
 bin/ext/cli.sh                                  |    8 -
 bin/ext/debug.cmd                               |  110 -
 bin/ext/hbaseimport.cmd                         |   35 -
 bin/ext/help.cmd                                |   30 -
 bin/ext/hiveserver2.cmd                         |  139 -
 bin/ext/jar.cmd                                 |   43 -
 bin/ext/lineage.cmd                             |   30 -
 bin/ext/metastore.cmd                           |   47 -
 bin/ext/orcfiledump.cmd                         |   35 -
 bin/ext/rcfilecat.cmd                           |   34 -
 bin/ext/schemaTool.cmd                          |   33 -
 bin/ext/util/execHiveCmd.cmd                    |   24 -
 bin/hive                                        |    7 +
 bin/hive-config.cmd                             |   61 -
 bin/hive.cmd                                    |  383 --
 bin/hplsql.cmd                                  |   58 -
 .../hadoop/hive/cli/TestCliDriverMethods.java   |   28 -
 .../apache/hadoop/hive/common/FileUtils.java    |    9 -
 .../hadoop/hive/common/StatsSetupConst.java     |  223 +-
 .../hadoop/hive/common/log/InPlaceUpdate.java   |    1 +
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   61 +-
 .../common/util/ACLConfigurationParser.java     |  167 +
 .../java/org/apache/hive/http/HttpServer.java   |    2 +-
 .../hadoop/hive/common/TestStatsSetupConst.java |   54 +
 .../apache/hadoop/hive/conf/TestHiveConf.java   |    5 -
 .../common/util/TestACLConfigurationParser.java |   99 +
 .../hadoop/hive/contrib/mr/TestGenericMR.java   |   13 +-
 data/conf/hive-site.xml                         |    2 +-
 druid-handler/pom.xml                           |   33 +-
 .../hadoop/hive/druid/DruidStorageHandler.java  |    7 +-
 .../hive/druid/DruidStorageHandlerUtils.java    |   64 +-
 .../druid/io/DruidQueryBasedInputFormat.java    |  198 +-
 .../hadoop/hive/druid/io/HiveDruidSplit.java    |   30 +-
 .../druid/serde/DruidQueryRecordReader.java     |    3 +-
 .../hadoop/hive/druid/serde/DruidSerDe.java     |   18 +-
 .../TestHiveDruidQueryBasedInputFormat.java     |  159 +
 .../hive/ql/io/DruidRecordWriterTest.java       |    2 +-
 hcatalog/bin/templeton.cmd                      |   90 -
 .../hive/hcatalog/mapreduce/HCatBaseTest.java   |    4 -
 .../mapreduce/TestHCatPartitionPublish.java     |   11 +-
 .../pig/TestHCatLoaderComplexSchema.java        |    5 -
 .../hcatalog/pig/TestHCatLoaderEncryption.java  |   11 +-
 .../listener/DbNotificationListener.java        |    1 +
 .../e2e/templeton/drivers/TestDriverCurl.pm     | 1984 ++++----
 .../hive/hcatalog/api/TestHCatClient.java       |    7 -
 .../hcatalog/templeton/ExecServiceImpl.java     |   48 +-
 .../hive/hcatalog/templeton/HiveDelegator.java  |    9 +-
 .../hive/hcatalog/templeton/JarDelegator.java   |    9 +-
 .../hive/hcatalog/templeton/PigDelegator.java   |    5 +-
 .../hive/hcatalog/templeton/SqoopDelegator.java |    7 +-
 .../hcatalog/templeton/StreamingDelegator.java  |    4 +-
 .../hcatalog/templeton/tool/LaunchMapper.java   |    9 -
 .../hcatalog/templeton/tool/TempletonUtils.java |   72 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |    8 +-
 .../listener/TestDbNotificationListener.java    |   82 +
 itests/hive-blobstore/pom.xml                   |    6 -
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |    5 +-
 .../hive/ql/TestReplicationScenarios.java       |  123 +-
 .../security/StorageBasedMetastoreTestBase.java |    4 -
 .../ql/session/TestClearDanglingScratchDir.java |    4 -
 .../hive/beeline/TestBeeLineWithArgs.java       |  169 +-
 .../TestOperationLoggingAPIWithTez.java         |    2 -
 .../server/TestHS2ClearDanglingScratchDir.java  |    4 -
 itests/qtest-accumulo/pom.xml                   |    6 -
 itests/qtest-spark/pom.xml                      |    6 -
 itests/qtest/pom.xml                            |    6 -
 .../test/resources/testconfiguration.properties |    7 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   55 +-
 jdbc/pom.xml                                    |   23 +-
 .../org/apache/hive/jdbc/HiveBaseResultSet.java |  297 +-
 .../apache/hive/jdbc/HiveCallableStatement.java |  421 +-
 .../org/apache/hive/jdbc/HiveConnection.java    |   64 +-
 .../org/apache/hive/jdbc/HiveDataSource.java    |   12 +-
 .../apache/hive/jdbc/HiveDatabaseMetaData.java  |  215 +-
 .../apache/hive/jdbc/HivePreparedStatement.java |   73 +-
 .../apache/hive/jdbc/HiveQueryResultSet.java    |    7 +-
 .../apache/hive/jdbc/HiveResultSetMetaData.java |   19 +-
 .../org/apache/hive/jdbc/HiveStatement.java     |   32 +-
 .../org/apache/hive/jdbc/HiveStatementTest.java |   14 +-
 .../llap/registry/impl/LlapRegistryService.java |   13 +-
 .../impl/LlapZookeeperRegistryImpl.java         |   11 +-
 .../hive/llap/security/SecretManager.java       |   19 +-
 .../org/apache/hadoop/hive/llap/LlapDump.java   |   30 +-
 .../hive/llap/cache/SerDeLowLevelCacheImpl.java |   13 +-
 .../hive/llap/cli/LlapOptionsProcessor.java     |   28 +-
 .../hadoop/hive/llap/cli/LlapServiceDriver.java |  290 +-
 .../hadoop/hive/llap/cli/LlapSliderUtils.java   |  188 +
 .../hive/llap/cli/LlapStatusServiceDriver.java  |   15 +-
 .../hive/llap/daemon/impl/AMReporter.java       |    5 +-
 .../llap/daemon/impl/ContainerRunnerImpl.java   |   16 +-
 .../hive/llap/daemon/impl/LlapDaemon.java       |   57 +-
 .../hadoop/hive/llap/daemon/impl/QueryInfo.java |   29 +
 .../hive/llap/daemon/impl/QueryTracker.java     |    4 +-
 .../llap/daemon/impl/TaskRunnerCallable.java    |   31 +-
 .../hive/llap/io/api/impl/LlapRecordReader.java |   11 +-
 .../llap/io/decode/OrcEncodedDataConsumer.java  |   22 +-
 .../llap/io/encoded/OrcEncodedDataReader.java   |    6 +-
 .../llap/io/encoded/SerDeEncodedDataReader.java |  882 ++--
 .../io/encoded/VectorDeserializeOrcWriter.java  |  458 ++
 .../io/encoded/VertorDeserializeOrcWriter.java  |  261 --
 .../llap/shufflehandler/ShuffleHandler.java     |    5 +-
 .../resources/llap-daemon-log4j2.properties     |   14 +-
 llap-server/src/main/resources/package.py       |   29 +-
 llap-server/src/main/resources/templates.py     |    3 +-
 .../hive/llap/daemon/MiniLlapCluster.java       |   38 +-
 .../daemon/impl/TaskExecutorTestHelpers.java    |    4 +-
 .../llap/tezplugins/LlapTaskCommunicator.java   |   81 +-
 .../tezplugins/LlapTaskSchedulerService.java    |   16 +
 .../tezplugins/helpers/SourceStateTracker.java  |   44 +-
 metastore/if/hive_metastore.thrift              |    1 +
 .../upgrade/derby/038-HIVE-10562.derby.sql      |   11 +
 .../upgrade/derby/hive-schema-2.2.0.derby.sql   |    2 +-
 .../derby/upgrade-2.1.0-to-2.2.0.derby.sql      |    1 +
 .../upgrade/mssql/023-HIVE-10562.mssql.sql      |    1 +
 .../upgrade/mssql/hive-schema-2.2.0.mssql.sql   |    1 +
 .../mssql/upgrade-2.1.0-to-2.2.0.mssql.sql      |    1 +
 .../upgrade/mysql/038-HIVE-10562.mysql.sql      |    6 +
 .../upgrade/mysql/hive-schema-2.2.0.mysql.sql   |    3 +-
 .../mysql/upgrade-2.1.0-to-2.2.0.mysql.sql      |    1 +
 .../upgrade/oracle/038-HIVE-10562.oracle.sql    |    2 +
 .../upgrade/oracle/hive-schema-2.2.0.oracle.sql |    3 +-
 .../oracle/upgrade-2.1.0-to-2.2.0.oracle.sql    |    1 +
 .../postgres/037-HIVE-10562.postgres.sql        |    1 +
 .../postgres/hive-schema-2.2.0.postgres.sql     |    1 +
 .../upgrade-2.1.0-to-2.2.0.postgres.sql         |    1 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |   22 +
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   12 +-
 .../hive/metastore/api/NotificationEvent.java   |  114 +-
 .../src/gen/thrift/gen-php/metastore/Types.php  |   23 +
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   15 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    4 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |   38 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    9 +-
 .../hadoop/hive/metastore/ObjectStore.java      |    2 +
 .../hive/metastore/RetryingMetaStoreClient.java |    4 +-
 .../hive/metastore/messaging/EventUtils.java    |   28 +-
 .../metastore/messaging/MessageFactory.java     |    5 -
 .../messaging/json/JSONMessageFactory.java      |    7 +-
 .../hive/metastore/model/MNotificationLog.java  |    9 +
 metastore/src/model/package.jdo                 |    3 +
 pom.xml                                         |   50 +-
 ql/pom.xml                                      |    4 +-
 .../FilterColumnBetweenDynamicValue.txt         |   11 +-
 .../java/org/apache/hadoop/hive/ql/Context.java |  121 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |    9 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |    3 +
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |   10 +-
 .../hadoop/hive/ql/exec/OperatorUtils.java      |   34 +
 .../hadoop/hive/ql/exec/ScriptOperator.java     |   17 -
 .../apache/hadoop/hive/ql/exec/Utilities.java   |   38 +-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java    |    6 +-
 .../persistence/BytesBytesMultiHashMap.java     |   10 +-
 .../hive/ql/exec/spark/GroupByShuffler.java     |   11 +-
 .../hive/ql/exec/spark/HiveReduceFunction.java  |   10 +-
 .../spark/HiveReduceFunctionResultList.java     |   18 +-
 .../ql/exec/spark/HiveSparkClientFactory.java   |    5 +
 .../hadoop/hive/ql/exec/spark/ReduceTran.java   |    8 +-
 .../hive/ql/exec/spark/RepartitionShuffler.java |   42 +
 .../hive/ql/exec/spark/SortByShuffler.java      |    2 +-
 .../hive/ql/exec/spark/SparkPlanGenerator.java  |    6 +-
 .../ql/exec/spark/SparkReduceRecordHandler.java |   56 +-
 .../hive/ql/exec/spark/SparkShuffler.java       |    4 +-
 .../spark/status/RemoteSparkJobMonitor.java     |    7 +-
 .../ql/exec/spark/status/SparkJobMonitor.java   |   12 +
 .../spark/status/impl/RemoteSparkJobStatus.java |    8 +
 .../hive/ql/exec/tez/TezJobExecHelper.java      |   29 +-
 .../hive/ql/exec/tez/TezSessionPoolManager.java |    2 +-
 .../hive/ql/exec/tez/TezSessionState.java       |   27 +
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |   45 +-
 .../apache/hadoop/hive/ql/exec/tez/Utils.java   |    6 +-
 .../hive/ql/exec/tez/monitoring/DAGSummary.java |   12 +-
 .../ql/exec/tez/monitoring/RenderStrategy.java  |  154 +
 .../ql/exec/tez/monitoring/TezJobMonitor.java   |  115 +-
 .../ql/exec/vector/VectorizationContext.java    |  114 +-
 .../fast/VectorMapJoinFastBytesHashMap.java     |    4 +-
 .../VectorMapJoinFastBytesHashMultiSet.java     |    4 +-
 .../fast/VectorMapJoinFastBytesHashSet.java     |    4 +-
 .../fast/VectorMapJoinFastBytesHashTable.java   |    8 +-
 .../mapjoin/fast/VectorMapJoinFastHashMap.java  |    4 +-
 .../fast/VectorMapJoinFastHashMultiSet.java     |    4 +-
 .../mapjoin/fast/VectorMapJoinFastHashSet.java  |    4 +-
 .../fast/VectorMapJoinFastHashTable.java        |   20 +-
 .../fast/VectorMapJoinFastLongHashMap.java      |    4 +-
 .../fast/VectorMapJoinFastLongHashMultiSet.java |    4 +-
 .../fast/VectorMapJoinFastLongHashSet.java      |    4 +-
 .../fast/VectorMapJoinFastLongHashTable.java    |    8 +-
 .../fast/VectorMapJoinFastMultiKeyHashMap.java  |    4 +-
 .../VectorMapJoinFastMultiKeyHashMultiSet.java  |    4 +-
 .../fast/VectorMapJoinFastMultiKeyHashSet.java  |    4 +-
 .../fast/VectorMapJoinFastStringHashMap.java    |    4 +-
 .../VectorMapJoinFastStringHashMultiSet.java    |    4 +-
 .../fast/VectorMapJoinFastStringHashSet.java    |    4 +-
 .../fast/VectorMapJoinFastTableContainer.java   |   27 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |   17 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   39 +
 .../hive/ql/io/orc/encoded/CacheChunk.java      |    3 +-
 .../orc/encoded/EncodedTreeReaderFactory.java   |  497 ++-
 .../hadoop/hive/ql/io/orc/encoded/Reader.java   |   69 +-
 .../vector/VectorizedParquetRecordReader.java   |    8 +-
 .../hadoop/hive/ql/lockmgr/DbLockManager.java   |   28 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |  144 +-
 .../metadata/HiveMaterializedViewsRegistry.java |    3 +-
 .../hive/ql/metadata/HiveMetaStoreChecker.java  |   45 +-
 .../AnnotateReduceSinkOutputOperator.java       |   73 +
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |  112 +-
 .../DynamicPartitionPruningOptimization.java    |   36 +-
 .../hadoop/hive/ql/optimizer/Optimizer.java     |    4 +
 .../SparkRemoveDynamicPruningBySize.java        |    4 +-
 .../calcite/CalciteViewSemanticException.java   |   52 +
 .../calcite/rules/HiveRelDecorrelator.java      |  724 +--
 .../calcite/translator/ASTBuilder.java          |   38 +-
 .../translator/SqlFunctionConverter.java        |   33 +-
 .../index/RewriteParseContextGenerator.java     |    4 +-
 .../spark/SetSparkReducerParallelism.java       |   79 +-
 .../optimizer/spark/SparkMapJoinOptimizer.java  |   34 +-
 .../stats/annotation/StatsRulesProcFactory.java |   44 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  147 +-
 .../ql/parse/ColumnStatsAutoGatherContext.java  |    4 +-
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |    4 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   11 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  162 +-
 .../apache/hadoop/hive/ql/parse/ParseUtils.java |  190 +-
 .../org/apache/hadoop/hive/ql/parse/QB.java     |    4 +
 .../ql/parse/ReplicationSemanticAnalyzer.java   |    9 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   47 +-
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |   62 +-
 .../hive/ql/parse/spark/GenSparkUtils.java      |   60 +-
 .../hive/ql/parse/spark/SparkCompiler.java      |   26 +-
 .../hadoop/hive/ql/plan/ReduceSinkDesc.java     |   14 +
 .../hadoop/hive/ql/session/SessionState.java    |   36 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |    6 +-
 .../hadoop/hive/ql/tools/LineageInfo.java       |    5 +-
 .../hive/ql/txn/compactor/CompactorMR.java      |    2 +-
 .../ql/udf/generic/GenericUDAFBloomFilter.java  |   28 +-
 .../generic/GenericUDAFPercentileApprox.java    |    2 +
 .../hive/ql/udf/generic/GenericUDFBasePad.java  |    2 +-
 .../hive/ql/udf/generic/GenericUDFLpad.java     |    9 +-
 .../hive/ql/udf/generic/GenericUDFRpad.java     |    8 +-
 .../ql/udf/generic/GenericUDFSQCountCheck.java  |   11 +-
 .../apache/hadoop/hive/ql/util/DosToUnix.java   |  107 -
 .../hadoop/hive/ql/util/ResourceDownloader.java |   12 +-
 .../apache/hadoop/hive/ql/TestTxnCommands.java  |   22 +
 .../apache/hadoop/hive/ql/WindowsPathUtil.java  |   57 -
 .../hadoop/hive/ql/exec/TestExecDriver.java     |    8 +-
 .../hadoop/hive/ql/exec/tez/TestTezTask.java    |    6 +
 .../exec/vector/TestVectorizationContext.java   |   37 +
 .../fast/TestVectorMapJoinFastBytesHashMap.java |   56 +-
 .../TestVectorMapJoinFastBytesHashMultiSet.java |   14 +-
 .../fast/TestVectorMapJoinFastBytesHashSet.java |   14 +-
 .../fast/TestVectorMapJoinFastLongHashMap.java  |   50 +-
 .../TestVectorMapJoinFastLongHashMultiSet.java  |   14 +-
 .../fast/TestVectorMapJoinFastLongHashSet.java  |   14 +-
 .../fast/TestVectorMapJoinFastRowHashMap.java   |   48 +-
 .../io/parquet/TestVectorizedColumnReader.java  |   21 +
 .../parquet/VectorizedColumnReaderTestBase.java |    2 +-
 .../ql/metadata/TestHiveMetaStoreChecker.java   |    4 -
 .../ql/parse/TestMacroSemanticAnalyzer.java     |    4 +-
 .../hive/ql/parse/TestParseDriverIntervals.java |    2 +-
 .../hadoop/hive/ql/parse/TestQBSubQuery.java    |    2 +-
 .../parse/TestUpdateDeleteSemanticAnalyzer.java |    4 +-
 .../authorization/AuthorizationTestUtil.java    |    2 +-
 .../hadoop/hive/ql/session/TestAddResource.java |    8 +-
 .../hive/ql/txn/compactor/CompactorTest.java    |   11 +-
 .../hive/ql/txn/compactor/TestCleaner.java      |    6 +
 .../hive/ql/txn/compactor/TestInitiator.java    |    5 +
 .../hive/ql/txn/compactor/TestWorker.java       |    6 +
 .../hive/ql/udf/generic/TestGenericUDFLpad.java |    9 +-
 .../hive/ql/udf/generic/TestGenericUDFRpad.java |   11 +-
 .../hadoop/hive/ql/util/TestDosToUnix.java      |   77 -
 .../clientnegative/masking_acid_delete.q        |   10 +
 .../queries/clientnegative/masking_acid_merge.q |   15 +
 .../clientnegative/masking_acid_update.q        |   10 +
 .../clientnegative/subquery_corr_in_agg.q       |    8 +
 ql/src/test/queries/clientnegative/udf_in_2.q   |    2 +
 .../queries/clientpositive/avro_timestamp.q     |    2 -
 .../queries/clientpositive/avro_timestamp_win.q |   28 -
 .../queries/clientpositive/cbo_rp_auto_join1.q  |    2 +-
 ql/src/test/queries/clientpositive/combine2.q   |    3 -
 .../queries/clientpositive/combine2_hadoop20.q  |    3 -
 .../test/queries/clientpositive/combine2_win.q  |   41 -
 .../dynamic_semijoin_reduction_2.q              |   41 +
 ql/src/test/queries/clientpositive/escape1.q    |    3 -
 ql/src/test/queries/clientpositive/escape2.q    |    3 -
 .../clientpositive/explain_formatted_oid.q      |   18 +
 .../test/queries/clientpositive/input_part10.q  |    3 -
 .../queries/clientpositive/input_part10_win.q   |   23 -
 .../test/queries/clientpositive/interval_alt.q  |   19 +-
 .../clientpositive/interval_arithmetic.q        |    3 +
 ql/src/test/queries/clientpositive/join31.q     |    2 +-
 .../queries/clientpositive/join_max_hashtable.q |   37 +
 .../clientpositive/lateral_view_explode2.q      |    4 +-
 .../test/queries/clientpositive/llap_reader.q   |   40 +
 .../queries/clientpositive/load_dyn_part14.q    |    3 -
 .../clientpositive/load_dyn_part14_win.q        |   38 -
 ql/src/test/queries/clientpositive/masking_10.q |   25 +
 .../clientpositive/msck_repair_batchsize.q      |   10 +
 .../test/queries/clientpositive/multiMapJoin2.q |    2 +-
 ql/src/test/queries/clientpositive/parenthese.q |   80 +
 .../clientpositive/partition_timestamp.q        |    2 -
 .../clientpositive/partition_timestamp2.q       |    2 -
 .../clientpositive/partition_timestamp2_win.q   |   58 -
 .../clientpositive/partition_timestamp_win.q    |   59 -
 .../clientpositive/position_alias_test_1.q      |   18 +
 .../test/queries/clientpositive/scriptfile1.q   |    2 -
 .../queries/clientpositive/scriptfile1_win.q    |   16 -
 .../spark_use_file_size_for_mapjoin.q           |   30 +
 .../queries/clientpositive/spark_use_op_stats.q |   41 +
 ql/src/test/queries/clientpositive/sqlmerge.q   |    6 +
 .../queries/clientpositive/subquery_multi.q     |    8 +-
 .../queries/clientpositive/transform_acid.q     |    2 -
 .../clientpositive/udaf_percentile_approx_23.q  |    3 +
 ql/src/test/queries/clientpositive/udf_in.q     |    4 +-
 ql/src/test/queries/clientpositive/udf_notop.q  |    2 +-
 .../queries/clientpositive/union_pos_alias.q    |   30 +
 .../vector_partitioned_date_time.q              |    3 -
 .../vector_partitioned_date_time_win.q          |  129 -
 .../vectorized_dynamic_semijoin_reduction.q     |    6 +
 .../vectorized_dynamic_semijoin_reduction2.q    |   65 +
 ql/src/test/queries/clientpositive/view_cbo.q   |   72 +
 .../clientnegative/char_pad_convert_fail2.q.out |    2 +-
 .../create_or_replace_view4.q.out               |    2 +-
 .../clientnegative/create_view_failure3.q.out   |    2 +-
 .../clientnegative/create_view_failure6.q.out   |    2 +-
 .../clientnegative/create_view_failure7.q.out   |    2 +-
 .../clientnegative/create_view_failure8.q.out   |    2 +-
 .../clientnegative/create_view_failure9.q.out   |    2 +-
 .../invalid_select_expression.q.out             |    2 +-
 .../clientnegative/masking_acid_delete.q.out    |   13 +
 .../clientnegative/masking_acid_merge.q.out     |   21 +
 .../clientnegative/masking_acid_update.q.out    |   13 +
 .../ptf_negative_DistributeByOrderBy.q.out      |    2 +-
 .../ptf_negative_PartitionBySortBy.q.out        |    2 +-
 .../clientnegative/ptf_window_boundaries.q.out  |    2 +-
 .../clientnegative/ptf_window_boundaries2.q.out |    2 +-
 .../selectDistinctStarNeg_1.q.out               |    2 +-
 .../clientnegative/select_charliteral.q.out     |    3 +-
 .../clientnegative/subquery_corr_in_agg.q.out   |   39 +
 .../clientnegative/subquery_missing_from.q.out  |    2 +-
 .../subquery_subquery_chain.q.out               |    2 +-
 .../clientnegative/subquery_with_or_cond.q.out  |    2 +-
 ql/src/test/results/clientnegative/udf_in.q.out |    2 +-
 .../test/results/clientnegative/udf_in_2.q.out  |    1 +
 .../clientpositive/acid_globallimit.q.out       |   52 -
 .../avro_timestamp_win.q.java1.7.out            |  134 -
 .../avro_timestamp_win.q.java1.8.out            |  134 -
 .../results/clientpositive/combine2_win.q.out   |  767 ----
 .../clientpositive/constant_prop_1.q.out        |    4 +-
 .../clientpositive/constprog_partitioner.q.out  |  117 +-
 .../results/clientpositive/create_view.q.out    |   10 +-
 .../clientpositive/create_view_translate.q.out  |    4 +-
 ql/src/test/results/clientpositive/cte_2.q.out  |    4 +-
 ql/src/test/results/clientpositive/cte_4.q.out  |    4 +-
 ...on_join_with_different_encryption_keys.q.out |   14 +-
 .../results/clientpositive/explain_ddl.q.out    |    4 +-
 .../clientpositive/explain_dependency.q.out     |    2 +-
 .../clientpositive/explain_formatted_oid.q.out  |   38 +
 .../clientpositive/explain_logical.q.out        |    2 +-
 ql/src/test/results/clientpositive/input4.q.out |    2 +-
 .../clientpositive/input_part10_win.q.out       |  131 -
 .../results/clientpositive/interval_alt.q.out   |   79 +-
 .../clientpositive/interval_arithmetic.q.out    |   35 +
 ql/src/test/results/clientpositive/join0.q.out  |    2 +-
 .../results/clientpositive/llap/cbo_views.q.out |    2 +-
 .../results/clientpositive/llap/cte_2.q.out     |    4 +-
 .../results/clientpositive/llap/cte_4.q.out     |    4 +-
 .../llap/dynamic_semijoin_reduction_2.q.out     |  301 ++
 .../clientpositive/llap/explainuser_1.q.out     |  688 +--
 .../llap/join_max_hashtable.q.out               |  490 ++
 .../results/clientpositive/llap/lineage3.q.out  |    2 +-
 .../clientpositive/llap/llap_reader.q.out       |  167 +
 .../results/clientpositive/llap/mergejoin.q.out |    8 +-
 .../llap/metadata_only_queries.q.out            |    4 +-
 ...chema_evol_text_vec_part_all_primitive.q.out |   34 +-
 .../llap/selectDistinctStar.q.out               |    8 +-
 .../results/clientpositive/llap/sqlmerge.q.out  |  101 +
 .../clientpositive/llap/subquery_exists.q.out   |  169 +-
 .../clientpositive/llap/subquery_in.q.out       | 2838 +++---------
 .../clientpositive/llap/subquery_multi.q.out    | 2788 +++---------
 .../clientpositive/llap/subquery_notin.q.out    | 4222 +++++-------------
 .../clientpositive/llap/subquery_scalar.q.out   | 2679 +++--------
 .../clientpositive/llap/subquery_views.q.out    |  558 +--
 .../clientpositive/llap/union_top_level.q.out   |    4 +-
 .../clientpositive/llap/vector_between_in.q.out |   22 +-
 .../llap/vector_decimal_mapjoin.q.out           |    4 +-
 .../llap/vector_mapjoin_reduce.q.out            |  196 +-
 .../llap/vector_reduce_groupby_decimal.q.out    |    2 +-
 .../vectorized_dynamic_semijoin_reduction.q.out |  141 +
 ...vectorized_dynamic_semijoin_reduction2.q.out | 1192 +++++
 .../results/clientpositive/llap_reader.q.out    |   86 +
 .../clientpositive/load_dyn_part14_win.q.out    |  298 --
 .../results/clientpositive/masking_10.q.out     |  244 +
 .../clientpositive/metadata_only_queries.q.out  |    4 +-
 ql/src/test/results/clientpositive/mm_all.q.out |   10 +-
 .../clientpositive/msck_repair_batchsize.q.out  |   25 +
 .../results/clientpositive/parallel_join0.q.out |    2 +-
 .../results/clientpositive/parenthese.q.out     |  168 +
 .../partition_timestamp2_win.q.out              |  399 --
 .../partition_timestamp_win.q.out               |  316 --
 .../results/clientpositive/perf/query1.q.out    |  190 +-
 .../results/clientpositive/perf/query14.q.out   |    8 +-
 .../results/clientpositive/perf/query16.q.out   |  222 +-
 .../results/clientpositive/perf/query23.q.out   |    4 +-
 .../results/clientpositive/perf/query30.q.out   |  375 +-
 .../results/clientpositive/perf/query6.q.out    |  403 +-
 .../results/clientpositive/perf/query69.q.out   |  499 +--
 .../results/clientpositive/perf/query81.q.out   |  375 +-
 .../test/results/clientpositive/plan_json.q.out |    2 +-
 .../clientpositive/position_alias_test_1.q.out  |  148 +
 .../clientpositive/scriptfile1_win.q.out        |   55 -
 .../test/results/clientpositive/semijoin5.q.out |  138 +-
 .../spark/constprog_partitioner.q.out           |   87 +-
 .../spark/metadata_only_queries.q.out           |    4 +-
 .../spark/spark_use_file_size_for_mapjoin.q.out |  257 ++
 .../spark/spark_use_op_stats.q.out              |  331 ++
 .../clientpositive/spark/subquery_exists.q.out  |  167 +-
 .../clientpositive/spark/subquery_in.q.out      | 2597 +++--------
 .../clientpositive/spark/union_remove_25.q.out  |    2 +-
 .../clientpositive/spark/union_top_level.q.out  |    4 +-
 .../spark/vector_between_in.q.out               |   22 +-
 .../spark/vector_decimal_mapjoin.q.out          |    4 +-
 .../spark/vector_mapjoin_reduce.q.out           |  216 +-
 .../clientpositive/subquery_exists.q.out        |  221 +-
 .../clientpositive/subquery_exists_having.q.out |  232 +-
 .../clientpositive/subquery_in_having.q.out     |  641 +--
 .../clientpositive/subquery_notexists.q.out     |  209 +-
 .../subquery_notexists_having.q.out             |  223 +-
 .../clientpositive/subquery_notin_having.q.out  |  644 +--
 .../subquery_unqualcolumnrefs.q.out             |  687 +--
 .../clientpositive/tez/explainanalyze_3.q.out   |    4 +-
 .../clientpositive/tez/explainuser_3.q.out      |    4 +-
 .../udaf_percentile_approx_23.q.out             |    9 +
 .../results/clientpositive/udf_between.q.out    |    2 +-
 ql/src/test/results/clientpositive/udf_in.q.out |    4 +-
 .../test/results/clientpositive/udf_lpad.q.out  |    6 +-
 .../test/results/clientpositive/udf_notop.q.out |    4 +-
 .../test/results/clientpositive/udf_rpad.q.out  |    6 +-
 .../clientpositive/union_pos_alias.q.out        |  308 ++
 .../clientpositive/vector_decimal_mapjoin.q.out |    2 +-
 .../clientpositive/vector_mapjoin_reduce.q.out  |  322 +-
 .../clientpositive/vector_outer_join3.q.out     |    6 +-
 .../clientpositive/vector_outer_join4.q.out     |    6 +-
 .../clientpositive/vector_outer_join6.q.out     |    4 +-
 .../vector_partitioned_date_time_win.q.out      | 2036 ---------
 .../vector_reduce_groupby_decimal.q.out         |    2 +-
 .../results/clientpositive/view_alias.q.out     |   12 +-
 .../test/results/clientpositive/view_cbo.q.out  |  823 ++++
 .../lazy/fast/LazySimpleDeserializeRead.java    |    2 +-
 .../gen/thrift/gen-cpp/TCLIService_types.cpp    |    8 +-
 .../src/gen/thrift/gen-cpp/TCLIService_types.h  |   11 +-
 .../service/rpc/thrift/TOpenSessionReq.java     |    4 +-
 .../service/rpc/thrift/TOpenSessionResp.java    |    4 +-
 .../service/rpc/thrift/TProtocolVersion.java    |    5 +-
 service-rpc/src/gen/thrift/gen-php/Types.php    |    6 +-
 .../src/gen/thrift/gen-py/TCLIService/ttypes.py |    7 +-
 .../gen/thrift/gen-rb/t_c_l_i_service_types.rb  |    9 +-
 .../org/apache/hive/service/cli/CLIService.java |   45 +-
 .../hive/service/cli/operation/Operation.java   |   16 +-
 .../service/cli/operation/SQLOperation.java     |    4 +-
 .../service/cli/thrift/ThriftCLIService.java    |   24 +-
 .../cli/thrift/ThriftHttpCLIService.java        |    2 +-
 .../apache/hadoop/fs/ProxyLocalFileSystem.java  |    8 -
 .../org/apache/hive/spark/client/rpc/Rpc.java   |    2 +-
 .../common/io/encoded/EncodedColumnBatch.java   |    1 +
 .../hive/ql/exec/vector/BytesColumnVector.java  |   11 +
 .../hive/ql/exec/vector/ColumnVector.java       |   12 +
 .../ql/exec/vector/DecimalColumnVector.java     |   11 +-
 .../hive/ql/exec/vector/DoubleColumnVector.java |    8 +-
 .../vector/IntervalDayTimeColumnVector.java     |    8 +
 .../hive/ql/exec/vector/LongColumnVector.java   |    8 +-
 .../ql/exec/vector/MultiValuedColumnVector.java |    4 +
 .../hive/ql/exec/vector/StructColumnVector.java |    5 +
 .../ql/exec/vector/TimestampColumnVector.java   |    8 +
 .../hive/ql/exec/vector/UnionColumnVector.java  |    5 +
 .../hive/serde2/io/HiveDecimalWritable.java     |    5 +-
 testutils/hadoop.cmd                            |  252 --
 vector-code-gen/pom.xml                         |   69 +
 .../apache/hadoop/hive/tools/GenVectorCode.java | 3327 ++++++++++++++
 .../hadoop/hive/tools/GenVectorTestCode.java    |  261 ++
 496 files changed, 23684 insertions(+), 31973 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/derby/hive-schema-2.2.0.derby.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
----------------------------------------------------------------------
diff --cc metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
index f59b37f,e5a144c..cbf5be1
--- a/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.1.0-to-2.2.0.derby.sql
@@@ -1,6 -1,5 +1,7 @@@
  -- Upgrade MetaStore schema from 2.1.0 to 2.2.0
  RUN '037-HIVE-14496.derby.sql';
+ RUN '038-HIVE-10562.derby.sql';
  
 +RUN '037-HIVE-14637.derby.sql';
 +
  UPDATE "APP".VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/mssql/hive-schema-2.2.0.mssql.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
----------------------------------------------------------------------
diff --cc metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
index 2dcdedc,a4b8fda..99024c2
--- a/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-2.1.0-to-2.2.0.mssql.sql
@@@ -1,7 -1,7 +1,8 @@@
  SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0' AS MESSAGE;
  
  :r 022-HIVE-14496.mssql.sql
 +:r 023-HIVE-14637.mssql.sql
+ :r 023-HIVE-10562.mssql.sql
  
  UPDATE VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;
  SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/mysql/hive-schema-2.2.0.mysql.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
----------------------------------------------------------------------
diff --cc metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
index afe17f9,509c532..68300d3
--- a/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-2.1.0-to-2.2.0.mysql.sql
@@@ -1,7 -1,7 +1,8 @@@
  SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0' AS ' ';
  
  SOURCE 037-HIVE-14496.mysql.sql;
 +SOURCE 038-HIVE-14637.mysql.sql;
+ SOURCE 038-HIVE-10562.mysql.sql;
  
  UPDATE VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;
  SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/oracle/hive-schema-2.2.0.oracle.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
----------------------------------------------------------------------
diff --cc metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
index dd1b97c,f31fda9..058c0d5
--- a/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-2.1.0-to-2.2.0.oracle.sql
@@@ -1,7 -1,7 +1,8 @@@
  SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0' AS Status from dual;
  
  @037-HIVE-14496.oracle.sql;
 +@038-HIVE-14637.oracle.sql;
+ @038-HIVE-10562.oracle.sql;
  
  UPDATE VERSION SET SCHEMA_VERSION='2.2.0', VERSION_COMMENT='Hive release version 2.2.0' where VER_ID=1;
  SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/postgres/hive-schema-2.2.0.postgres.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
----------------------------------------------------------------------
diff --cc metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
index a316383,0f64a90..ae4adf7
--- a/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-2.1.0-to-2.2.0.postgres.sql
@@@ -1,7 -1,7 +1,8 @@@
  SELECT 'Upgrading MetaStore schema from 2.1.0 to 2.2.0';
  
  \i 036-HIVE-14496.postgres.sql;
 +\i 037-HIVE-14637.postgres.sql;
+ \i 037-HIVE-10562.postgres.sql;
  
  UPDATE "VERSION" SET "SCHEMA_VERSION"='2.2.0', "VERSION_COMMENT"='Hive release version 2.2.0' where "VER_ID"=1;
  SELECT 'Finished upgrading MetaStore schema from 2.1.0 to 2.2.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/metastore/src/model/package.jdo
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/ql/src/java/org/apache/hadoop/hive/ql/Context.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index d7db991,3484493..966b2b5
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@@ -200,9 -141,9 +200,10 @@@ import org.apache.hadoop.mapred.RecordR
  import org.apache.hadoop.mapred.Reporter;
  import org.apache.hadoop.mapred.SequenceFileInputFormat;
  import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 +import org.apache.hadoop.mapred.TextInputFormat;
  import org.apache.hadoop.util.Progressable;
  import org.apache.hadoop.util.Shell;
+ import org.apache.hive.common.util.ACLConfigurationParser;
  import org.apache.hive.common.util.ReflectionUtil;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
@@@ -3966,331 -3796,25 +3951,352 @@@ public final class Utilities 
      return String.format("%.2f%sB", bytes / Math.pow(unit, exp), suffix);
    }
  
 +  private static final String MANIFEST_EXTENSION = ".manifest";
 +
 +  private static void tryDelete(FileSystem fs, Path path) {
 +    try {
 +      fs.delete(path, true);
 +    } catch (IOException ex) {
 +      LOG.error("Failed to delete " + path, ex);
 +    }
 +  }
 +
 +  public static Path[] getMmDirectoryCandidates(FileSystem fs, Path path, int dpLevels,
 +      int lbLevels, PathFilter filter, long mmWriteId, Configuration conf) throws IOException {
 +    int skipLevels = dpLevels + lbLevels;
 +    if (filter == null) {
 +      filter = new ValidWriteIds.IdPathFilter(mmWriteId, true);
 +    }
 +    if (skipLevels == 0) {
 +      return statusToPath(fs.listStatus(path, filter));
 +    }
 +    if (HiveConf.getBoolVar(conf, ConfVars.HIVE_MM_AVOID_GLOBSTATUS_ON_S3) && isS3(fs)) {
 +      return getMmDirectoryCandidatesRecursive(fs, path, skipLevels, filter);
 +    }
 +    return getMmDirectoryCandidatesGlobStatus(fs, path, skipLevels, filter, mmWriteId);
 +  }
 +
 +  private static boolean isS3(FileSystem fs) {
 +    try {
 +      return fs.getScheme().equalsIgnoreCase("s3a");
 +    } catch (UnsupportedOperationException ex) {
 +      // Some FS-es do not implement getScheme, e.g. ProxyLocalFileSystem.
 +      return false;
 +    }
 +  }
 +
 +  private static Path[] statusToPath(FileStatus[] statuses) {
 +    if (statuses == null) return null;
 +    Path[] paths = new Path[statuses.length];
 +    for (int i = 0; i < statuses.length; ++i) {
 +      paths[i] = statuses[i].getPath();
 +    }
 +    return paths;
 +  }
 +
 +  private static Path[] getMmDirectoryCandidatesRecursive(FileSystem fs,
 +      Path path, int skipLevels, PathFilter filter) throws IOException {
 +    String lastRelDir = null;
 +    HashSet<Path> results = new HashSet<Path>();
 +    String relRoot = Path.getPathWithoutSchemeAndAuthority(path).toString();
 +    if (!relRoot.endsWith(Path.SEPARATOR)) {
 +      relRoot += Path.SEPARATOR;
 +    }
 +    RemoteIterator<LocatedFileStatus> allFiles = fs.listFiles(path, true);
 +    while (allFiles.hasNext()) {
 +      LocatedFileStatus lfs = allFiles.next();
 +      Path dirPath = Path.getPathWithoutSchemeAndAuthority(lfs.getPath());
 +      String dir = dirPath.toString();
 +      if (!dir.startsWith(relRoot)) {
 +        throw new IOException("Path " + lfs.getPath() + " is not under " + relRoot
 +            + " (when shortened to " + dir + ")");
 +      }
 +      String subDir = dir.substring(relRoot.length());
 +      Utilities.LOG14535.info("Looking at " + subDir + " from " + lfs.getPath());
 +      // If sorted, we'll skip a bunch of files.
 +      if (lastRelDir != null && subDir.startsWith(lastRelDir)) continue;
 +      int startIx = skipLevels > 0 ? -1 : 0;
 +      for (int i = 0; i < skipLevels; ++i) {
 +        startIx = subDir.indexOf(Path.SEPARATOR_CHAR, startIx + 1);
 +        if (startIx == -1) {
 +          Utilities.LOG14535.info("Expected level of nesting (" + skipLevels + ") is not "
 +              + " present in " + subDir + " (from " + lfs.getPath() + ")");
 +          break;
 +        }
 +      }
 +      if (startIx == -1) continue;
 +      int endIx = subDir.indexOf(Path.SEPARATOR_CHAR, startIx + 1);
 +      if (endIx == -1) {
 +        Utilities.LOG14535.info("Expected level of nesting (" + (skipLevels + 1) + ") is not "
 +            + " present in " + subDir + " (from " + lfs.getPath() + ")");
 +        continue;
 +      }
 +      lastRelDir = subDir = subDir.substring(0, endIx);
 +      Path candidate = new Path(relRoot, subDir);
 +      Utilities.LOG14535.info("Considering MM directory candidate " + candidate);
 +      if (!filter.accept(candidate)) continue;
 +      results.add(fs.makeQualified(candidate));
 +    }
 +    return results.toArray(new Path[results.size()]);
 +  }
 +
 +  private static Path[] getMmDirectoryCandidatesGlobStatus(FileSystem fs,
 +      Path path, int skipLevels, PathFilter filter, long mmWriteId) throws IOException {
 +    StringBuilder sb = new StringBuilder(path.toUri().getPath());
 +    for (int i = 0; i < skipLevels; i++) {
 +      sb.append(Path.SEPARATOR).append("*");
 +    }
 +    sb.append(Path.SEPARATOR).append(ValidWriteIds.getMmFilePrefix(mmWriteId));
 +    Path pathPattern = new Path(path, sb.toString());
 +    Utilities.LOG14535.info("Looking for files via: " + pathPattern);
 +    return statusToPath(fs.globStatus(pathPattern, filter));
 +  }
 +
 +  private static void tryDeleteAllMmFiles(FileSystem fs, Path specPath, Path manifestDir,
 +      int dpLevels, int lbLevels, String unionSuffix, ValidWriteIds.IdPathFilter filter,
 +      long mmWriteId, Configuration conf) throws IOException {
 +    Path[] files = getMmDirectoryCandidates(
 +        fs, specPath, dpLevels, lbLevels, filter, mmWriteId, conf);
 +    if (files != null) {
 +      for (Path path : files) {
 +        Utilities.LOG14535.info("Deleting " + path + " on failure");
 +        tryDelete(fs, path);
 +      }
 +    }
 +    Utilities.LOG14535.info("Deleting " + manifestDir + " on failure");
 +    fs.delete(manifestDir, true);
 +  }
 +
 +
 +  public static void writeMmCommitManifest(List<Path> commitPaths, Path specPath, FileSystem fs,
 +      String taskId, Long mmWriteId, String unionSuffix) throws HiveException {
 +    if (commitPaths.isEmpty()) return;
 +    // We assume one FSOP per task (per specPath), so we create it in specPath.
 +    Path manifestPath = getManifestDir(specPath, mmWriteId, unionSuffix);
 +    manifestPath = new Path(manifestPath, taskId + MANIFEST_EXTENSION);
 +    Utilities.LOG14535.info("Writing manifest to " + manifestPath + " with " + commitPaths);
 +    try {
 +      // Don't overwrite the manifest... should fail if we have collisions.
 +      try (FSDataOutputStream out = fs.create(manifestPath, false)) {
 +        if (out == null) {
 +          throw new HiveException("Failed to create manifest at " + manifestPath);
 +        }
 +        out.writeInt(commitPaths.size());
 +        for (Path path : commitPaths) {
 +          out.writeUTF(path.toString());
 +        }
 +      }
 +    } catch (IOException e) {
 +      throw new HiveException(e);
 +    }
 +  }
 +
 +  private static Path getManifestDir(Path specPath, long mmWriteId, String unionSuffix) {
 +    Path manifestPath = new Path(specPath, "_tmp." + ValidWriteIds.getMmFilePrefix(mmWriteId));
 +    return (unionSuffix == null) ? manifestPath : new Path(manifestPath, unionSuffix);
 +  }
 +
 +  public static final class MissingBucketsContext {
 +    public final TableDesc tableInfo;
 +    public final int numBuckets;
 +    public final boolean isCompressed;
 +    public MissingBucketsContext(TableDesc tableInfo, int numBuckets, boolean isCompressed) {
 +      this.tableInfo = tableInfo;
 +      this.numBuckets = numBuckets;
 +      this.isCompressed = isCompressed;
 +    }
 +  }
 +
 +  public static void handleMmTableFinalPath(Path specPath, String unionSuffix, Configuration hconf,
 +      boolean success, int dpLevels, int lbLevels, MissingBucketsContext mbc, long mmWriteId,
 +      Reporter reporter, boolean isMmCtas) throws IOException, HiveException {
 +    FileSystem fs = specPath.getFileSystem(hconf);
 +    Path manifestDir = getManifestDir(specPath, mmWriteId, unionSuffix);
 +    if (!success) {
 +      ValidWriteIds.IdPathFilter filter = new ValidWriteIds.IdPathFilter(mmWriteId, true);
 +      tryDeleteAllMmFiles(fs, specPath, manifestDir, dpLevels, lbLevels,
 +          unionSuffix, filter, mmWriteId, hconf);
 +      return;
 +    }
 +
 +    Utilities.LOG14535.info("Looking for manifests in: " + manifestDir + " (" + mmWriteId + ")");
 +    // TODO# may be wrong if there are no splits (empty insert/CTAS)
 +    List<Path> manifests = new ArrayList<>();
 +    if (fs.exists(manifestDir)) {
 +      FileStatus[] manifestFiles = fs.listStatus(manifestDir);
 +      if (manifestFiles != null) {
 +        for (FileStatus status : manifestFiles) {
 +          Path path = status.getPath();
 +          if (path.getName().endsWith(MANIFEST_EXTENSION)) {
 +            Utilities.LOG14535.info("Reading manifest " + path);
 +            manifests.add(path);
 +          }
 +        }
 +      }
 +    } else {
 +      Utilities.LOG14535.info("No manifests found - query produced no output");
 +      manifestDir = null;
 +    }
 +
 +    Utilities.LOG14535.info("Looking for files in: " + specPath);
 +    ValidWriteIds.IdPathFilter filter = new ValidWriteIds.IdPathFilter(mmWriteId, true);
 +    if (isMmCtas && !fs.exists(specPath)) {
 +      // TODO: do we also need to do this when creating an empty partition from select?
 +      Utilities.LOG14535.info("Creating table directory for CTAS with no output at " + specPath);
 +      FileUtils.mkdir(fs, specPath, hconf);
 +    }
 +    Path[] files = getMmDirectoryCandidates(
 +        fs, specPath, dpLevels, lbLevels, filter, mmWriteId, hconf);
 +    ArrayList<Path> mmDirectories = new ArrayList<>();
 +    if (files != null) {
 +      for (Path path : files) {
 +        Utilities.LOG14535.info("Looking at path: " + path);
 +        mmDirectories.add(path);
 +      }
 +    }
 +
 +    HashSet<String> committed = new HashSet<>();
 +    for (Path mfp : manifests) {
 +      try (FSDataInputStream mdis = fs.open(mfp)) {
 +        int fileCount = mdis.readInt();
 +        for (int i = 0; i < fileCount; ++i) {
 +          String nextFile = mdis.readUTF();
 +          if (!committed.add(nextFile)) {
 +            throw new HiveException(nextFile + " was specified in multiple manifests");
 +          }
 +        }
 +      }
 +    }
 +
 +    if (manifestDir != null) {
 +      Utilities.LOG14535.info("Deleting manifest directory " + manifestDir);
 +      tryDelete(fs, manifestDir);
 +      if (unionSuffix != null) {
 +        // Also delete the parent directory if we are the last union FSOP to execute.
 +        manifestDir = manifestDir.getParent();
 +        FileStatus[] remainingFiles = fs.listStatus(manifestDir);
 +        if (remainingFiles == null || remainingFiles.length == 0) {
 +          Utilities.LOG14535.info("Deleting manifest directory " + manifestDir);
 +          tryDelete(fs, manifestDir);
 +        }
 +      }
 +    }
 +
 +    for (Path path : mmDirectories) {
 +      cleanMmDirectory(path, fs, unionSuffix, committed);
 +    }
 +
 +    if (!committed.isEmpty()) {
 +      throw new HiveException("The following files were committed but not found: " + committed);
 +    }
 +
 +    if (mmDirectories.isEmpty()) return;
 +
 +    // TODO: see HIVE-14886 - removeTempOrDuplicateFiles is broken for list bucketing,
 +    //       so maintain parity here by not calling it at all.
 +    if (lbLevels != 0) return;
 +    // Create fake file statuses to avoid querying the file system. removeTempOrDuplicateFiles
 +    // doesn't need tocheck anything except path and directory status for MM directories.
 +    FileStatus[] finalResults = new FileStatus[mmDirectories.size()];
 +    for (int i = 0; i < mmDirectories.size(); ++i) {
 +      finalResults[i] = new PathOnlyFileStatus(mmDirectories.get(i));
 +    }
 +    List<Path> emptyBuckets = Utilities.removeTempOrDuplicateFiles(
 +        fs, finalResults, dpLevels, mbc == null ? 0 : mbc.numBuckets, hconf, mmWriteId);
 +    // create empty buckets if necessary
 +    if (emptyBuckets.size() > 0) {
 +      assert mbc != null;
 +      Utilities.createEmptyBuckets(hconf, emptyBuckets, mbc.isCompressed, mbc.tableInfo, reporter);
 +    }
 +  }
 +
 +  private static final class PathOnlyFileStatus extends FileStatus {
 +    public PathOnlyFileStatus(Path path) {
 +      super(0, true, 0, 0, 0, path);
 +    }
 +  }
 +
 +  private static void cleanMmDirectory(Path dir, FileSystem fs,
 +      String unionSuffix, HashSet<String> committed) throws IOException, HiveException {
 +    for (FileStatus child : fs.listStatus(dir)) {
 +      Path childPath = child.getPath();
 +      if (unionSuffix == null) {
 +        if (committed.remove(childPath.toString())) continue; // A good file.
 +        deleteUncommitedFile(childPath, fs);
 +      } else if (!child.isDirectory()) {
 +        if (committed.contains(childPath.toString())) {
 +          throw new HiveException("Union FSOP has commited "
 +              + childPath + " outside of union directory" + unionSuffix);
 +        }
 +        deleteUncommitedFile(childPath, fs);
 +      } else if (childPath.getName().equals(unionSuffix)) {
 +        // Found the right union directory; treat it as "our" MM directory.
 +        cleanMmDirectory(childPath, fs, null, committed);
 +      } else {
 +        Utilities.LOG14535.info("FSOP for " + unionSuffix
 +            + " is ignoring the other side of the union " + childPath.getName());
 +      }
 +    }
 +  }
 +
 +  private static void deleteUncommitedFile(Path childPath, FileSystem fs)
 +      throws IOException, HiveException {
 +    Utilities.LOG14535.info("Deleting " + childPath + " that was not committed");
 +    // We should actually succeed here - if we fail, don't commit the query.
 +    if (!fs.delete(childPath, true)) {
 +      throw new HiveException("Failed to delete an uncommitted path " + childPath);
 +    }
 +  }
 +
 +  /**
 +   * @return the complete list of valid MM directories under a table/partition path; null
 +   * if the entire directory is valid (has no uncommitted/temporary files).
 +   */
 +  public static List<Path> getValidMmDirectoriesFromTableOrPart(Path path, Configuration conf,
 +      ValidWriteIds ids, int lbLevels) throws IOException {
 +    Utilities.LOG14535.info("Looking for valid MM paths under " + path);
 +    // NULL means this directory is entirely valid.
 +    List<Path> result = null;
 +    FileSystem fs = path.getFileSystem(conf);
 +    FileStatus[] children = (lbLevels == 0) ? fs.listStatus(path)
 +        : fs.globStatus(new Path(path, StringUtils.repeat("*" + Path.SEPARATOR, lbLevels) + "*"));
 +    for (int i = 0; i < children.length; ++i) {
 +      FileStatus file = children[i];
 +      Path childPath = file.getPath();
 +      Long writeId = ValidWriteIds.extractWriteId(childPath);
 +      if (!file.isDirectory() || writeId == null || !ids.isValid(writeId)) {
 +        Utilities.LOG14535.info("Skipping path " + childPath);
 +        if (result == null) {
 +          result = new ArrayList<>(children.length - 1);
 +          for (int j = 0; j < i; ++j) {
 +            result.add(children[j].getPath());
 +          }
 +        }
 +      } else if (result != null) {
 +        result.add(childPath);
 +      }
 +    }
 +    return result;
 +  }
+ 
+   public static String getAclStringWithHiveModification(Configuration tezConf,
+                                                         String propertyName,
+                                                         boolean addHs2User,
+                                                         String user,
+                                                         String hs2User) throws
+       IOException {
+ 
+     // Start with initial ACLs
+     ACLConfigurationParser aclConf =
+         new ACLConfigurationParser(tezConf, propertyName);
+ 
+     // Always give access to the user
+     aclConf.addAllowedUser(user);
+ 
+     // Give access to the process user if the config is set.
+     if (addHs2User && hs2User != null) {
+       aclConf.addAllowedUser(hs2User);
+     }
+     return aclConf.toAclString();
+   }
  }

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2014ece9/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
----------------------------------------------------------------------


[02/50] [abbrv] hive git commit: HIVE-15900 : Beeline prints tez job progress in stdout instead of stderr (Thejas Nair, reviewed by Daniel Dai, Anishek Agarwal)

Posted by se...@apache.org.
HIVE-15900 : Beeline prints tez job progress in stdout instead of stderr (Thejas Nair, reviewed by Daniel Dai, Anishek Agarwal)


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

Branch: refs/heads/hive-14535
Commit: 60a36d124d437bea123c94d50ccb171e24ff2f3d
Parents: 6e652a3
Author: Thejas M Nair <th...@hortonworks.com>
Authored: Thu Feb 16 11:43:45 2017 -0800
Committer: Thejas M Nair <th...@hortonworks.com>
Committed: Thu Feb 16 11:43:52 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/Commands.java  |   2 +-
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   |   2 +
 .../hive/beeline/TestBeeLineWithArgs.java       | 169 +++++++++++++------
 .../TestOperationLoggingAPIWithTez.java         |   2 -
 4 files changed, 118 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/60a36d12/beeline/src/java/org/apache/hive/beeline/Commands.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java
index 99db643..6a3ad42 100644
--- a/beeline/src/java/org/apache/hive/beeline/Commands.java
+++ b/beeline/src/java/org/apache/hive/beeline/Commands.java
@@ -985,7 +985,7 @@ public class Commands {
             logThread.start();
             if (stmnt instanceof HiveStatement) {
               ((HiveStatement) stmnt).setInPlaceUpdateStream(
-                  new BeelineInPlaceUpdateStream(beeLine.getOutputStream())
+                  new BeelineInPlaceUpdateStream(beeLine.getErrorStream())
               );
             }
             hasResults = stmnt.execute(sql);

http://git-wip-us.apache.org/repos/asf/hive/blob/60a36d12/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
index e641253..71f9640 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -222,6 +222,8 @@ public class MiniHS2 extends AbstractHiveService {
       // Initialize the execution engine based on cluster type
       switch (miniClusterType) {
       case TEZ:
+        // Change the engine to tez
+        hiveConf.setVar(ConfVars.HIVE_EXECUTION_ENGINE, "tez");
         // TODO: This should be making use of confDir to load configs setup for Tez, etc.
         mr = ShimLoader.getHadoopShims().getMiniTezCluster(hiveConf, 2, uriString, false);
         break;

http://git-wip-us.apache.org/repos/asf/hive/blob/60a36d12/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
index 9e99a91..8fe3789 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
@@ -36,11 +36,16 @@ import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.jdbc.Utils;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -51,11 +56,15 @@ import org.junit.Test;
  *
  */
 public class TestBeeLineWithArgs {
+  private enum OutStream {
+    ERR, OUT
+  };
+
   // Default location of HiveServer2
   private static final String tableName = "TestBeelineTable1";
   private static final String tableComment = "Test table comment";
-
   private static MiniHS2 miniHS2;
+  private static final String userName = System.getProperty("user.name");
 
   private List<String> getBaseArgs(String jdbcUrl) {
     List<String> argList = new ArrayList<String>(8);
@@ -63,6 +72,8 @@ public class TestBeeLineWithArgs {
     argList.add(BeeLine.BEELINE_DEFAULT_JDBC_DRIVER);
     argList.add("-u");
     argList.add(jdbcUrl);
+    argList.add("-n");
+    argList.add(userName);
     return argList;
   }
   /**
@@ -71,11 +82,15 @@ public class TestBeeLineWithArgs {
   @BeforeClass
   public static void preTests() throws Exception {
     HiveConf hiveConf = new HiveConf();
-    // Set to non-zk lock manager to prevent HS2 from trying to connect
-    hiveConf.setVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER, "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager");
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER,
+        "org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager");
     hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES, false);
-    miniHS2 = new MiniHS2(hiveConf);
-    miniHS2.start(new HashMap<String,  String>());
+    hiveConf.set(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname, "verbose");
+    miniHS2 = new MiniHS2(hiveConf, MiniClusterType.TEZ);
+
+    Map<String, String> confOverlay = new HashMap<String, String>();
+    miniHS2.start(confOverlay);
+
     createTable();
   }
 
@@ -86,7 +101,8 @@ public class TestBeeLineWithArgs {
    */
   private static void createTable() throws ClassNotFoundException, SQLException {
     Class.forName(BeeLine.BEELINE_DEFAULT_JDBC_DRIVER);
-    Connection con = DriverManager.getConnection(miniHS2.getBaseJdbcURL(),"", "");
+    Connection con = DriverManager.getConnection(miniHS2.getBaseJdbcURL(),
+        userName , "");
 
     assertNotNull("Connection is null", con);
     assertFalse("Connection should not be closed", con.isClosed());
@@ -128,16 +144,27 @@ public class TestBeeLineWithArgs {
 
   /**
    * Execute a script with "beeline -f or -i"
-   *
+   * @param argList List of arguments for beeline
+   * @param inputStream input stream if any
+   * @param streamType if output from STDERR or STDOUT needs to be returned
    * @return The stderr and stdout from running the script
+   * @throws Throwable
    */
-  private String testCommandLineScript(List<String> argList, InputStream inputStream)
+  private String testCommandLineScript(List<String> argList, InputStream inputStream, OutStream streamType)
       throws Throwable {
     BeeLine beeLine = new BeeLine();
     ByteArrayOutputStream os = new ByteArrayOutputStream();
     PrintStream beelineOutputStream = new PrintStream(os);
-    beeLine.setOutputStream(beelineOutputStream);
-    beeLine.setErrorStream(beelineOutputStream);
+    switch (streamType) {
+    case OUT:
+      beeLine.setOutputStream(beelineOutputStream);
+      break;
+    case ERR:
+      beeLine.setErrorStream(beelineOutputStream);
+      break;
+    default:
+      throw new RuntimeException("Unexpected outstream type " + streamType);
+    }
     String[] args = argList.toArray(new String[argList.size()]);
     beeLine.begin(args, inputStream);
     String output = os.toString("UTF8");
@@ -147,33 +174,53 @@ public class TestBeeLineWithArgs {
   }
 
   /**
+   * Attempt to execute a simple script file with the -f and -i option to
+   * BeeLine to test for presence of an expected pattern in the output (stdout
+   * or stderr), fail if not found. Print PASSED or FAILED
+   * 
+   * @param expectedRegex
+   *          Text to look for in command output (stdout)
+   * @param shouldMatch
+   *          true if the pattern should be found, false if it should not
+   * @throws Exception
+   *           on command execution error
+   */
+  private void testScriptFile(String scriptText, String expectedRegex,
+      boolean shouldMatch, List<String> argList) throws Throwable {
+    testScriptFile(scriptText, expectedRegex, shouldMatch, argList, true, true, OutStream.OUT);
+  }
+
+  /**
    * Attempt to execute a simple script file with the -f and -i option
    * to BeeLine to test for presence of an expected pattern
    * in the output (stdout or stderr), fail if not found.
    * Print PASSED or FAILED
-   * @param expectedPattern Text to look for in command output/error
+   * @param expectedRegex Text to look for in command output (stdout)
    * @param shouldMatch true if the pattern should be found, false if it should not
-   * @throws Exception on command execution error
+   * @param argList arguments
+   * @param outType output stream type
+   * @throws Throwable
    */
-  private void testScriptFile(String scriptText, String expectedPattern,
-      boolean shouldMatch, List<String> argList) throws Throwable {
-    testScriptFile(scriptText, expectedPattern, shouldMatch, argList, true, true);
+  private void testScriptFile(String scriptText, String expectedRegex,
+      boolean shouldMatch, List<String> argList, OutStream outType) throws Throwable {
+    testScriptFile(scriptText, expectedRegex, shouldMatch, argList, true, true, outType);
   }
-
+  
   /**
    * Attempt to execute a simple script file with the -f or -i option
    * to BeeLine (or both) to  test for presence of an expected pattern
    * in the output (stdout or stderr), fail if not found.
    * Print PASSED or FAILED
-   * @param expectedPattern Text to look for in command output/error
+   * @param expectedRegex Text to look for in command output/error
    * @param shouldMatch true if the pattern should be found, false if it should not
    * @param testScript Whether we should test -f
    * @param testInit Whether we should test -i
+   * @param streamType Whether match should be done against STDERR or STDOUT
    * @throws Exception on command execution error
    */
-  private void testScriptFile(String scriptText, String expectedPattern,
+  private void testScriptFile(String scriptText, String expectedRegex,
       boolean shouldMatch, List<String> argList,
-      boolean testScript, boolean testInit) throws Throwable {
+      boolean testScript, boolean testInit, OutStream streamType) throws Throwable {
 
     // Put the script content in a temp file
     File scriptFile = File.createTempFile(this.getClass().getSimpleName(), "temp");
@@ -183,17 +230,20 @@ public class TestBeeLineWithArgs {
     os.print(scriptText);
     os.close();
 
+    Pattern expectedPattern = Pattern.compile(".*" + expectedRegex + ".*", Pattern.DOTALL);
     if (testScript) {
       List<String> copy = new ArrayList<String>(argList);
       copy.add("-f");
       copy.add(scriptFile.getAbsolutePath());
 
-      String output = testCommandLineScript(copy, null);
-      boolean matches = output.contains(expectedPattern);
+      String output = testCommandLineScript(copy, null, streamType);
+
+      Matcher m = expectedPattern.matcher(output);
+      boolean matches = m.matches();
       if (shouldMatch != matches) {
         //failed
         fail("Output" + output + " should" +  (shouldMatch ? "" : " not") +
-            " contain " + expectedPattern);
+            " contain " + expectedRegex);
       }
     }
 
@@ -205,12 +255,13 @@ public class TestBeeLineWithArgs {
       copy.add("-i");
       copy.add(scriptFile.getAbsolutePath());
 
-      String output = testCommandLineScript(copy, new StringBufferInputStream("!quit\n"));
-      boolean matches = output.contains(expectedPattern);
+      String output = testCommandLineScript(copy, new StringBufferInputStream("!quit\n"), streamType);
+      Matcher m = expectedPattern.matcher(output);
+      boolean matches = m.matches();
       if (shouldMatch != matches) {
         //failed
         fail("Output" + output + " should" +  (shouldMatch ? "" : " not") +
-            " contain " + expectedPattern);
+            " contain " + expectedRegex);
       }
     }
     scriptFile.delete();
@@ -225,14 +276,15 @@ public class TestBeeLineWithArgs {
    * @param shouldMatch true if the pattern should be found, false if it should not
    * @throws Exception on command execution error
    */
+
   private void testCommandEnclosedQuery(String enclosedQuery, String expectedPattern,
-      boolean shouldMatch, List<String> argList) throws Throwable {
+      boolean shouldMatch, List<String> argList, OutStream out) throws Throwable {
 
     List<String> copy = new ArrayList<String>(argList);
     copy.add("-e");
     copy.add(enclosedQuery);
 
-    String output = testCommandLineScript(copy, null);
+    String output = testCommandLineScript(copy, null, out);
     boolean matches = output.contains(expectedPattern);
     if (shouldMatch != matches) {
       //failed
@@ -290,7 +342,7 @@ public class TestBeeLineWithArgs {
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--hivevar");
     argList.add("DUMMY_TBL=dummy");
-    final String SCRIPT_TEXT = "create table ${DUMMY_TBL} (d int);\nshow tables;\n";
+    final String SCRIPT_TEXT = "create table ${DUMMY_TBL} (d int);\nshow tables;\n drop table  ${DUMMY_TBL};";
     final String EXPECTED_PATTERN = "dummy";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -300,7 +352,8 @@ public class TestBeeLineWithArgs {
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--hiveconf");
     argList.add("test.hive.table.name=dummy");
-    final String SCRIPT_TEXT = "create table ${hiveconf:test.hive.table.name} (d int);\nshow tables;\n";
+    final String SCRIPT_TEXT = "create table ${hiveconf:test.hive.table.name} (d int);\nshow tables;\n"
+        + " drop table ${hiveconf:test.hive.table.name};\n";
     final String EXPECTED_PATTERN = "dummy";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -327,7 +380,9 @@ public class TestBeeLineWithArgs {
     argList.add("--hiveconf");
     argList.add("COLUMN_TYPE=int");
 
-    final String SCRIPT_TEXT = "${COMMAND} ${OBJECT} ${TABLE_NAME} (${hiveconf:COLUMN_NAME} ${hiveconf:COLUMN_TYPE});\nshow tables;\n";
+    final String SCRIPT_TEXT = "${COMMAND} ${OBJECT} ${TABLE_NAME} "
+        + "(${hiveconf:COLUMN_NAME} ${hiveconf:COLUMN_TYPE});"
+        + "\nshow tables;\n drop ${OBJECT} ${TABLE_NAME};\n";
     final String EXPECTED_PATTERN = "dummy2";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -348,7 +403,8 @@ public class TestBeeLineWithArgs {
   @Test
   public void testTabInScriptFile() throws Throwable {
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
-    final String SCRIPT_TEXT = "CREATE\tTABLE IF NOT EXISTS testTabInScriptFile\n(id\tint);\nSHOW TABLES;";
+    final String SCRIPT_TEXT = "CREATE\tTABLE IF NOT EXISTS testTabInScriptFile\n(id\tint);\nSHOW TABLES;"
+        + "\ndrop table testTabInScriptFile";
     final String EXPECTED_PATTERN = "testTabInScriptFile";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -537,7 +593,7 @@ public class TestBeeLineWithArgs {
     argList.add("--outputformat=tsv");
 
     final String EXPECTED_PATTERN = "Format tsv is deprecated, please use tsv2";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, OutStream.ERR);
   }
 
   /**
@@ -551,7 +607,7 @@ public class TestBeeLineWithArgs {
     argList.add("--outputformat=csv");
 
     final String EXPECTED_PATTERN = "Format csv is deprecated, please use csv2";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, true, OutStream.ERR);
   }
 
   /**
@@ -612,7 +668,7 @@ public class TestBeeLineWithArgs {
     argList.add(scriptFile.getAbsolutePath());
 
     try {
-      String output = testCommandLineScript(argList, null);
+      String output = testCommandLineScript(argList, null, OutStream.OUT);
       if (output.contains(EXPECTED_PATTERN)) {
         fail("Output: " + output +  " Negative pattern: " + EXPECTED_PATTERN);
       }
@@ -651,7 +707,7 @@ public class TestBeeLineWithArgs {
   @Test
   public void testHiveVarSubstitution() throws Throwable {
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL() + "#D_TBL=dummy_t");
-    final String SCRIPT_TEXT = "create table ${D_TBL} (d int);\nshow tables;\n";
+    final String SCRIPT_TEXT = "create table ${D_TBL} (d int);\nshow tables;\ndrop  table ${D_TBL};\n";
     final String EXPECTED_PATTERN = "dummy_t";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -665,7 +721,7 @@ public class TestBeeLineWithArgs {
     // Set to non-zk lock manager to avoid trying to connect to zookeeper
     final String SCRIPT_TEXT =
         "set hive.lock.manager=org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager;\n" +
-        "create table ${DUMMY_TBL} (d int);\nshow tables;\n";
+        "create table ${DUMMY_TBL} (d int);\nshow tables;\n drop table ${DUMMY_TBL};\n";
     final String EXPECTED_PATTERN = "embedded_table";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
   }
@@ -678,8 +734,10 @@ public class TestBeeLineWithArgs {
   public void testQueryProgress() throws Throwable {
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
         "select count(*) from " + tableName + ";\n";
-    final String EXPECTED_PATTERN = "number of splits";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()));
+    // Check for part of log message as well as part of progress information
+    final String EXPECTED_PATTERN = "Number of reducers determined to be.*ELAPSED TIME";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()),
+        OutStream.ERR);
   }
 
   /**
@@ -692,8 +750,10 @@ public class TestBeeLineWithArgs {
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
         "set hive.exec.parallel = true;\n" +
         "select count(*) from " + tableName + ";\n";
-    final String EXPECTED_PATTERN = "number of splits";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()));
+    // Check for part of log message as well as part of progress information
+    final String EXPECTED_PATTERN = "Number of reducers determined to be.*ELAPSED TIME";
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()),
+        OutStream.ERR);
   }
 
   /**
@@ -706,7 +766,7 @@ public class TestBeeLineWithArgs {
         "!set silent true\n" +
         "select count(*) from " + tableName + ";\n";
     final String EXPECTED_PATTERN = "Executing command";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, false, getBaseArgs(miniHS2.getBaseJdbcURL()));
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, false, getBaseArgs(miniHS2.getBaseJdbcURL()), OutStream.ERR);
   }
 
   @Test
@@ -727,10 +787,10 @@ public class TestBeeLineWithArgs {
         +"(key int);show tables; --multicommands in one line";
     final String EXPECTED_PATTERN = " multicmdtbl ";
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
-    testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList);
+    testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList, OutStream.OUT);
 
     final String QUERY_TEXT_DROP = "drop table multiCmdTbl;show tables;";
-    testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList);
+    testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList, OutStream.OUT);
   }
 
   @Test
@@ -765,10 +825,10 @@ public class TestBeeLineWithArgs {
         + " TERMINATED BY '\\n';show tables;";
     final String EXPECTED_PATTERN = " multicmdtbl ";
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
-    testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList);
+    testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList, OutStream.OUT);
 
     final String QUERY_TEXT_DROP = "drop table multiCmdTbl;show tables;";
-    testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList);
+    testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList, OutStream.OUT);
   }
 
   @Test
@@ -779,9 +839,10 @@ public class TestBeeLineWithArgs {
     final String SCRIPT_TEXT = "set hive.lock.manager=org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager;\n"
         + "set hive.compute.query.using.stats=false;\n"
         + "create table if not exists embeddedBeelineOutputs(d int);\n"
-        + "set a=1;\nselect count(*) from embeddedBeelineOutputs;\n";
+        + "set a=1;\nselect count(*) from embeddedBeelineOutputs;\n"
+        + "drop table embeddedBeelineOutputs;\n";
     final String EXPECTED_PATTERN = "Stage-1 map =";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, OutStream.ERR);
   }
 
   @Test
@@ -806,7 +867,7 @@ public class TestBeeLineWithArgs {
     argList.add(BeeLine.BEELINE_DEFAULT_JDBC_DRIVER);
 
     final String SCRIPT_TEXT =
-        "create table blueconnecttest (d int);\nshow tables;\n";
+        "create table blueconnecttest (d int);\nshow tables;\ndrop table blueconnecttest;\n";
     final String EXPECTED_PATTERN = "blueconnecttest";
 
     // We go through these hijinxes because java considers System.getEnv
@@ -826,7 +887,7 @@ public class TestBeeLineWithArgs {
     };
     BeeLineOpts.setEnv(newEnv);
 
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false, OutStream.OUT);
   }
 
   /**
@@ -839,10 +900,10 @@ public class TestBeeLineWithArgs {
     final String SCRIPT_TEXT =
         "!close\n" +
         "!reconnect\n\n\n" +
-        "create table reconnecttest (d int);\nshow tables;\n";
+        "create table reconnecttest (d int);\nshow tables;\ndrop table reconnecttest;\n";
     final String EXPECTED_PATTERN = "reconnecttest";
 
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false, OutStream.OUT);
 
   }
 
@@ -894,7 +955,7 @@ public class TestBeeLineWithArgs {
    */
   @Test
   public void testShowDbInPrompt() throws Throwable {
-    final String EXPECTED_PATTERN = " (default)>";
+    final String EXPECTED_PATTERN = " \\(default\\)>";
     List<String> argList = new ArrayList<String>();
     argList.add("--showDbInPrompt");
     argList.add("-u");
@@ -909,7 +970,7 @@ public class TestBeeLineWithArgs {
     List<String> argList = new ArrayList<String>();
     final String SCRIPT_TEXT = "!sh echo hello world";
     final String EXPECTED_PATTERN = "hello world";
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList,true,false);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, true, false, OutStream.OUT);
   }
 
   /**
@@ -924,6 +985,6 @@ public class TestBeeLineWithArgs {
     final String EXPECTED_PATTERN = "2 rows selected";
     List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
     argList.add("--force");
-    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+    testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList, OutStream.ERR);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/60a36d12/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java
index e98406d..388486d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/cli/operation/TestOperationLoggingAPIWithTez.java
@@ -58,8 +58,6 @@ public class TestOperationLoggingAPIWithTez extends OperationLoggingAPITestBase
     };
     hiveConf = new HiveConf();
     hiveConf.set(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL.varname, "verbose");
-    // Change the engine to tez
-    hiveConf.setVar(ConfVars.HIVE_EXECUTION_ENGINE, "tez");
     // Set tez execution summary to false.
     hiveConf.setBoolVar(ConfVars.TEZ_EXEC_SUMMARY, false);
     miniHS2 = new MiniHS2(hiveConf, MiniClusterType.TEZ);


[05/50] [abbrv] hive git commit: HIVE-15877 : Upload dependency jars for druid storage handler (Slim Bouguerra via Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-15877 : Upload dependency jars for druid storage handler (Slim Bouguerra via Ashutosh Chauhan)

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


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

Branch: refs/heads/hive-14535
Commit: ef61a9bcfa90bd359e1af028af873fb0ccf51deb
Parents: f3790ce
Author: Slim Bouguerra <sl...@gmail.com>
Authored: Fri Feb 10 11:24:00 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Thu Feb 16 17:11:54 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hive/druid/DruidStorageHandler.java  |  7 ++-
 .../hive/druid/DruidStorageHandlerUtils.java    | 64 +++++++++++++++-----
 .../hive/metastore/HiveMetaStoreClient.java     |  5 +-
 3 files changed, 60 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ef61a9bc/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
index cff0056..d4f6865 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.druid.io.DruidOutputFormat;
 import org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat;
+import org.apache.hadoop.hive.druid.io.DruidRecordWriter;
 import org.apache.hadoop.hive.druid.serde.DruidSerDe;
 import org.apache.hadoop.hive.metastore.DefaultHiveMetaHook;
 import org.apache.hadoop.hive.metastore.HiveMetaHook;
@@ -520,7 +521,11 @@ public class DruidStorageHandler extends DefaultHiveMetaHook implements HiveStor
 
   @Override
   public void configureJobConf(TableDesc tableDesc, JobConf jobConf) {
-
+    try {
+      DruidStorageHandlerUtils.addDependencyJars(jobConf, DruidRecordWriter.class);
+    } catch (IOException e) {
+      Throwables.propagate(e);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/ef61a9bc/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
index 52e7e8d..8d48e14 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hive.druid;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.jsontype.NamedType;
 import com.fasterxml.jackson.dataformat.smile.SmileFactory;
-import com.google.common.base.Strings;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Interner;
@@ -28,13 +27,10 @@ import com.google.common.collect.Interners;
 import com.google.common.collect.Lists;
 import com.google.common.io.CharStreams;
 import com.metamx.common.MapUtils;
-import com.metamx.common.lifecycle.Lifecycle;
 import com.metamx.emitter.EmittingLogger;
 import com.metamx.emitter.core.NoopEmitter;
 import com.metamx.emitter.service.ServiceEmitter;
 import com.metamx.http.client.HttpClient;
-import com.metamx.http.client.HttpClientConfig;
-import com.metamx.http.client.HttpClientInit;
 import com.metamx.http.client.Request;
 import com.metamx.http.client.response.InputStreamResponseHandler;
 import io.druid.jackson.DefaultObjectMapper;
@@ -51,14 +47,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.util.StringUtils;
 import org.jboss.netty.handler.codec.http.HttpHeaders;
 import org.jboss.netty.handler.codec.http.HttpMethod;
-import org.joda.time.Period;
 import org.skife.jdbi.v2.FoldController;
 import org.skife.jdbi.v2.Folder3;
 import org.skife.jdbi.v2.Handle;
@@ -67,31 +62,46 @@ import org.skife.jdbi.v2.TransactionCallback;
 import org.skife.jdbi.v2.TransactionStatus;
 import org.skife.jdbi.v2.tweak.HandleCallback;
 import org.skife.jdbi.v2.util.ByteArrayMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.Reader;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.net.InetAddress;
-import java.net.URI;
 import java.net.URL;
+import java.net.URLDecoder;
 import java.net.UnknownHostException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.TimeZone;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+import static org.apache.hadoop.hive.ql.exec.Utilities.jarFinderGetJar;
 
 /**
  * Utils class for Druid storage handler.
  */
 public final class DruidStorageHandlerUtils {
 
+  private static final Logger LOG = LoggerFactory.getLogger(DruidStorageHandlerUtils.class);
+
   private static final String SMILE_CONTENT_TYPE = "application/x-jackson-smile";
+
   /**
    * Mapper to use to serialize/deserialize Druid objects (JSON)
    */
@@ -190,18 +200,17 @@ public final class DruidStorageHandlerUtils {
     return response;
   }
 
-
   public static String getURL(HttpClient client, URL url) throws IOException {
     try (Reader reader = new InputStreamReader(
             DruidStorageHandlerUtils.submitRequest(client, new Request(HttpMethod.GET, url)))) {
-      return  CharStreams.toString(reader);
+      return CharStreams.toString(reader);
     }
   }
 
   /**
    * @param taskDir path to the  directory containing the segments descriptor info
-   *                 the descriptor path will be .../workingPath/task_id/{@link DruidStorageHandler#SEGMENTS_DESCRIPTOR_DIR_NAME}/*.json
-   * @param conf     hadoop conf to get the file system
+   *                the descriptor path will be .../workingPath/task_id/{@link DruidStorageHandler#SEGMENTS_DESCRIPTOR_DIR_NAME}/*.json
+   * @param conf    hadoop conf to get the file system
    *
    * @return List of DataSegments
    *
@@ -290,7 +299,8 @@ public final class DruidStorageHandlerUtils {
                                   public ArrayList<String> fold(ArrayList<String> druidDataSources,
                                           Map<String, Object> stringObjectMap,
                                           FoldController foldController,
-                                          StatementContext statementContext) throws SQLException {
+                                          StatementContext statementContext
+                                  ) throws SQLException {
                                     druidDataSources.add(
                                             MapUtils.getString(stringObjectMap, "datasource")
                                     );
@@ -431,4 +441,30 @@ public final class DruidStorageHandlerUtils {
   public interface DataPusher {
     long push() throws IOException;
   }
+
+  // Thanks, HBase Storage handler
+  public static void addDependencyJars(Configuration conf, Class<?>... classes) throws IOException {
+    FileSystem localFs = FileSystem.getLocal(conf);
+    Set<String> jars = new HashSet<String>();
+    jars.addAll(conf.getStringCollection("tmpjars"));
+    for (Class<?> clazz : classes) {
+      if (clazz == null) {
+        continue;
+      }
+      String path = Utilities.jarFinderGetJar(clazz);
+      if (path == null) {
+        throw new RuntimeException(
+                "Could not find jar for class " + clazz + " in order to ship it to the cluster.");
+      }
+      if (!localFs.exists(new Path(path))) {
+        throw new RuntimeException("Could not validate jar file " + path + " for class " + clazz);
+      }
+      jars.add(path.toString());
+    }
+    if (jars.isEmpty()) {
+      return;
+    }
+    conf.set("tmpjars", StringUtils.arrayToString(jars.toArray(new String[jars.size()])));
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ef61a9bc/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index c32104f..70f3a6b 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -739,7 +739,10 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
         hook.commitCreateTable(tbl);
       }
       success = true;
-    } finally {
+    } catch (Exception e){
+      LOG.error("Got exception from createTable", e);
+    }
+    finally {
       if (!success && (hook != null)) {
         hook.rollbackCreateTable(tbl);
       }


[22/50] [abbrv] hive git commit: HIVE-15874: Invalid position alias in Group By when CBO failed (Walter Wu, reviewed by Pengcheng Xiong)

Posted by se...@apache.org.
HIVE-15874: Invalid position alias in Group By when CBO failed (Walter Wu, 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/08ca7b2d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/08ca7b2d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/08ca7b2d

Branch: refs/heads/hive-14535
Commit: 08ca7b2de8e50699b759d9b86eb8daaa59f580e9
Parents: 091ac8e
Author: Pengcheng Xiong <px...@apache.org>
Authored: Mon Feb 20 12:33:24 2017 -0800
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Mon Feb 20 12:33:24 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   2 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   8 +-
 ql/src/test/queries/clientpositive/masking_10.q |  25 ++
 .../clientpositive/position_alias_test_1.q      |  18 ++
 .../results/clientpositive/masking_10.q.out     | 244 +++++++++++++++++++
 .../clientpositive/position_alias_test_1.q.out  | 148 +++++++++++
 6 files changed, 443 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/08ca7b2d/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 e7687be..10f16ca 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
@@ -297,6 +297,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
   public RelNode genLogicalPlan(ASTNode ast) throws SemanticException {
     LOG.info("Starting generating logical plan");
     PreCboCtx cboCtx = new PreCboCtx();
+    //change the location of position alias process here
+    processPositionAlias(ast);
     if (!genResolvedParseTree(ast, cboCtx)) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/08ca7b2d/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 9c37af8..9eafb0b 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
@@ -10903,7 +10903,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     ctesExpanded = new ArrayList<String>();
 
     // 1. analyze and process the position alias
-    processPositionAlias(ast);
+    // step processPositionAlias out of genResolvedParseTree
 
     // 2. analyze create table command
     if (ast.getToken().getType() == HiveParser.TOK_CREATETABLE) {
@@ -11019,6 +11019,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   void analyzeInternal(ASTNode ast, PlannerContext plannerCtx) throws SemanticException {
     // 1. Generate Resolved Parse tree from syntax tree
     LOG.info("Starting Semantic Analysis");
+    //change the location of position alias process here
+    processPositionAlias(ast);
     if (!genResolvedParseTree(ast, plannerCtx)) {
       return;
     }
@@ -11032,6 +11034,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       if (tree != ast) {
         ctx.setSkipTableMasking(true);
         init(true);
+        //change the location of position alias process here
+        processPositionAlias(tree);
         genResolvedParseTree(tree, plannerCtx);
         if (this instanceof CalcitePlanner) {
           ((CalcitePlanner) this).resetCalciteConfiguration();
@@ -12258,7 +12262,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   // Process the position alias in GROUPBY and ORDERBY
-  private void processPositionAlias(ASTNode ast) throws SemanticException {
+  public void processPositionAlias(ASTNode ast) throws SemanticException {
     boolean isBothByPos = HiveConf.getBoolVar(conf, ConfVars.HIVE_GROUPBY_ORDERBY_POSITION_ALIAS);
     boolean isGbyByPos = isBothByPos
         || HiveConf.getBoolVar(conf, ConfVars.HIVE_GROUPBY_POSITION_ALIAS);

http://git-wip-us.apache.org/repos/asf/hive/blob/08ca7b2d/ql/src/test/queries/clientpositive/masking_10.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/masking_10.q b/ql/src/test/queries/clientpositive/masking_10.q
new file mode 100644
index 0000000..e933253
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/masking_10.q
@@ -0,0 +1,25 @@
+set hive.mapred.mode=nonstrict;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+
+drop table masking_test;	
+
+create temporary table masking_test as select cast(key as int) as key, value from src;
+	
+set hive.groupby.position.alias = true;
+set hive.cbo.enable=true;
+
+explain select 2017 as a, value from masking_test group by 1, 2;
+
+select 2017 as a, value from masking_test group by 1, 2;
+
+explain
+select * from
+  masking_test alias01
+  left join
+  (
+      select 2017 as a, value from masking_test group by 1, 2
+  ) alias02
+  on alias01.key = alias02.a
+  left join
+  masking_test alias03
+on alias01.key = alias03.key;

http://git-wip-us.apache.org/repos/asf/hive/blob/08ca7b2d/ql/src/test/queries/clientpositive/position_alias_test_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/position_alias_test_1.q b/ql/src/test/queries/clientpositive/position_alias_test_1.q
new file mode 100644
index 0000000..599bc08
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/position_alias_test_1.q
@@ -0,0 +1,18 @@
+create table alias_test_01(a INT, b STRING) ;
+                         create table alias_test_02(a INT, b STRING) ;
+                         create table alias_test_03(a INT, b STRING) ;
+                         set hive.groupby.position.alias = true;
+                         set hive.cbo.enable=true;
+
+
+                         explain
+                         select * from
+                         alias_test_01 alias01
+                         left join
+                         (
+                         select 2017 as a, b from alias_test_02 group by 1, 2
+                         ) alias02
+                         on alias01.a = alias02.a
+                         left join
+                         alias_test_03 alias03
+                         on alias01.a = alias03.a;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/08ca7b2d/ql/src/test/results/clientpositive/masking_10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/masking_10.q.out b/ql/src/test/results/clientpositive/masking_10.q.out
new file mode 100644
index 0000000..d6293e3
--- /dev/null
+++ b/ql/src/test/results/clientpositive/masking_10.q.out
@@ -0,0 +1,244 @@
+PREHOOK: query: drop table masking_test
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table masking_test
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create temporary table masking_test as select cast(key as int) as key, value from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@masking_test
+POSTHOOK: query: create temporary table masking_test as select cast(key as int) as key, value from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@masking_test
+PREHOOK: query: explain select 2017 as a, value from masking_test group by 1, 2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select 2017 as a, value from masking_test group by 1, 2
+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: masking_test
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((key % 2) = 0) and (key < 10)) (type: boolean)
+              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: reverse(value) (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  keys: _col0 (type: string)
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 83 Data size: 881 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: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: 2017 (type: int), _col0 (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 41 Data size: 435 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 2017 as a, value from masking_test group by 1, 2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test
+#### A masked pattern was here ####
+POSTHOOK: query: select 2017 as a, value from masking_test group by 1, 2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test
+#### A masked pattern was here ####
+2017	0_lav
+2017	2_lav
+2017	4_lav
+2017	8_lav
+Warning: Shuffle Join JOIN[34][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: explain
+select * from
+  masking_test alias01
+  left join
+  (
+      select 2017 as a, value from masking_test group by 1, 2
+  ) alias02
+  on alias01.key = alias02.a
+  left join
+  masking_test alias03
+on alias01.key = alias03.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select * from
+  masking_test alias01
+  left join
+  (
+      select 2017 as a, value from masking_test group by 1, 2
+  ) alias02
+  on alias01.key = alias02.a
+  left join
+  masking_test alias03
+on alias01.key = alias03.key
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-3 is a root stage
+  Stage-1 depends on stages: Stage-3
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: masking_test
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((key % 2) = 0) and (key < 10)) (type: boolean)
+              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: reverse(value) (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  keys: _col0 (type: string)
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 83 Data size: 881 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: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: 2017 (type: int), _col0 (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 41 Data size: 435 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: masking_test
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((key % 2) = 0) and (key < 10)) (type: boolean)
+              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: int), reverse(value) (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: int), _col1 (type: string)
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: int), _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          filter predicates:
+            0 {(VALUE._col0 = 2017)}
+            1 
+          keys:
+            0 
+            1 
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 3403 Data size: 75629 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: int)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 3403 Data size: 75629 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: string)
+          TableScan
+            alias: masking_test
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((key % 2) = 0) and (key < 10)) (type: boolean)
+              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: int), reverse(value) (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 83 Data size: 881 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: int)
+            1 _col0 (type: int)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 3743 Data size: 83191 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 3743 Data size: 83191 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
+

http://git-wip-us.apache.org/repos/asf/hive/blob/08ca7b2d/ql/src/test/results/clientpositive/position_alias_test_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/position_alias_test_1.q.out b/ql/src/test/results/clientpositive/position_alias_test_1.q.out
new file mode 100644
index 0000000..9053bf1
--- /dev/null
+++ b/ql/src/test/results/clientpositive/position_alias_test_1.q.out
@@ -0,0 +1,148 @@
+PREHOOK: query: create table alias_test_01(a INT, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alias_test_01
+POSTHOOK: query: create table alias_test_01(a INT, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alias_test_01
+PREHOOK: query: create table alias_test_02(a INT, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alias_test_02
+POSTHOOK: query: create table alias_test_02(a INT, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alias_test_02
+PREHOOK: query: create table alias_test_03(a INT, b STRING)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alias_test_03
+POSTHOOK: query: create table alias_test_03(a INT, b STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alias_test_03
+PREHOOK: query: explain
+                         select * from
+                         alias_test_01 alias01
+                         left join
+                         (
+                         select 2017 as a, b from alias_test_02 group by 1, 2
+                         ) alias02
+                         on alias01.a = alias02.a
+                         left join
+                         alias_test_03 alias03
+                         on alias01.a = alias03.a
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+                         select * from
+                         alias_test_01 alias01
+                         left join
+                         (
+                         select 2017 as a, b from alias_test_02 group by 1, 2
+                         ) alias02
+                         on alias01.a = alias02.a
+                         left join
+                         alias_test_03 alias03
+                         on alias01.a = alias03.a
+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: alias_test_02
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Select Operator
+              expressions: b (type: string)
+              outputColumnNames: b
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Group By Operator
+                keys: 2017 (type: int), b (type: string)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Reduce Output Operator
+                  key expressions: 2017 (type: int), _col1 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: 2017 (type: int), _col1 (type: string)
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+      Reduce Operator Tree:
+        Group By Operator
+          keys: 2017 (type: int), KEY._col1 (type: string)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: 2017 (type: int), _col1 (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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: int)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              value expressions: _col1 (type: string)
+          TableScan
+            alias: alias01
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Reduce Output Operator
+              key expressions: a (type: int)
+              sort order: +
+              Map-reduce partition columns: a (type: int)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              value expressions: b (type: string)
+          TableScan
+            alias: alias03
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Reduce Output Operator
+              key expressions: a (type: int)
+              sort order: +
+              Map-reduce partition columns: a (type: int)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              value expressions: b (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+               Left Outer Join0 to 2
+          keys:
+            0 a (type: int)
+            1 _col0 (type: int)
+            2 a (type: int)
+          outputColumnNames: _col0, _col1, _col5, _col6, _col7, _col8
+          Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: int), _col8 (type: string)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+            Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.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
+


[40/50] [abbrv] hive git commit: HIVE-15955: make explain formatted to include opId and etc (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/vector_outer_join3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_outer_join3.q.out b/ql/src/test/results/clientpositive/vector_outer_join3.q.out
index 49c658b..1d2abee 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join3.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join3.q.out
@@ -242,7 +242,7 @@ left outer join small_alltypesorc_a hd
   on hd.cstring1 = c.cstring1
 ) t1
 POSTHOOK: type: QUERY
-{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"cint (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: int)","1":"_col0 (type: int)"}}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE
 ","children":{"Select Operator":{"expressions:":"cstring1 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"}}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"children":{"Select Operator":{"expressions:":"cint (type: int), cstring1 (type: string)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[2, 6]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0
 ":"_col0 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col1"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true",
 "Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","Not ACID UPDATE or DELETE IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true
 ","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[2, 6]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Map Reduce Local Work":{}},"
 Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cint (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: int)","1":"_col0 (type: int)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statist
 ics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"cstring1 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"cint (type: int), cstring1 (type: string)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[2, 6]"},"Stat
 istics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col1"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:"
 :"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className
 :":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","Not ACID UPDATE or DELETE IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:"
 :"[2, 6]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Colum
 n 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}}
 PREHOOK: query: select count(*) from (select c.cstring1
 from small_alltypesorc_a c
 left outer join small_alltypesorc_a cd
@@ -282,7 +282,7 @@ left outer join small_alltypesorc_a hd
   on hd.cstring1 = c.cstring1
 ) t1
 POSTHOOK: type: QUERY
-{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"cstring2 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"}}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Colum
 n stats: NONE","children":{"Select Operator":{"expressions:":"cstring1 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"}}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"children":{"Select Operator":{"expressions:":"cstring1 (type: string), cstring2 (type: string)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[6, 7]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0
  to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS tr
 ue","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","Not ACID UPDATE or DELETE IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySort
 ableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[6, 7]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Ma
 p Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cstring2 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":
 "hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"cstring1 (type: string)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"cstring1 (type: string), cstring2 (type: string)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColum
 ns:":"[6, 7]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: string)","1":"_col0 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: string)","1":"_col0 (type: string)"},"Map Join Vec
 torization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink V
 ectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","Not ACID UPDATE or DELETE IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCoun
 t:":"12","includeColumns:":"[6, 7]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 B
 asic 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}}
 PREHOOK: query: select count(*) from (select c.cstring1
 from small_alltypesorc_a c
 left outer join small_alltypesorc_a cd
@@ -322,7 +322,7 @@ left outer join small_alltypesorc_a hd
   on hd.cstring1 = c.cstring1 and hd.cint = c.cint
 ) t1
 POSTHOOK: type: QUERY
-{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"cbigint (type: bigint), cstring2 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: bigint), _col3 (type: string)","1":"_col0 (type: bigint), _col1 (type: string)"}}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":
 "hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"cint (type: int), cstring1 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: int), _col2 (type: string)","1":"_col0 (type: int), _col1 (type: string)"}}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"children":{"Select Operator":{"expressions:":"cint (type: int), cbigint (type: bigint), cstring1 (type: string), cstring2 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"tr
 ue","projectedOutputColumns:":"[2, 3, 6, 7]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: bigint), _col3 (type: string)","1":"_col0 (type: bigint), _col1 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col2"],"Statistics:":"Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: int), _col2 (type
 : string)","1":"_col0 (type: int), _col1 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort orde
 r:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","Not ACID UPDATE or DELETE IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[2,
  3, 6, 7]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cbigint (type: bigint), cstring2 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: bigint), _col3 (type: string)","1":"_col0 (type: bigint), _col1 (type: string)"}
 ,"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"cint (type: int), cstring1 (type: string)","outputColumnNames:":["_col0","_col1"],"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: int), _col2 (type: string)","1":"_col0 (type: int), _col1 (type: string)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"cint (type: int), cbigint (type: bigint), cstrin
 g1 (type: string), cstring2 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[2, 3, 6, 7]"},"Statistics:":"Num rows: 20 Data size: 4400 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: bigint), _col3 (type: string)","1":"_col0 (type: bigint), _col1 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0","_col2"],"Statistics:":"
 Num rows: 22 Data size: 4840 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: int), _col2 (type: string)","1":"_col0 (type: int), _col1 (type: string)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 24 Data size: 5324 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator",
 "vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","Not ACID UPDATE or DELETE IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"tru
 e","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[2, 3, 6, 7]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":
 "false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}}
 PREHOOK: query: select count(*) from (select c.cstring1
 from small_alltypesorc_a c
 left outer join small_alltypesorc_a cd

http://git-wip-us.apache.org/repos/asf/hive/blob/759766ee/ql/src/test/results/clientpositive/vector_outer_join4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_outer_join4.q.out b/ql/src/test/results/clientpositive/vector_outer_join4.q.out
index fce35a1..a73a5e1 100644
--- a/ql/src/test/results/clientpositive/vector_outer_join4.q.out
+++ b/ql/src/test/results/clientpositive/vector_outer_join4.q.out
@@ -256,7 +256,7 @@ from small_alltypesorc_b c
 left outer join small_alltypesorc_b cd
   on cd.cint = c.cint
 POSTHOOK: type: QUERY
-{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-4":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-4"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-4":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)","outputColumnNames:":["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","
 _col10","_col11"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col2 (type: int)","1":"_col2 (type: int)"}}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)","outputColumnNames:":["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"],"Select Vectorization:":{"className:":
 "VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col2 (type: int)","1":"_col2 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"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"],"Statisti
 cs:":"Num rows: 33 Data size: 7348 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","File Sink Vectorization:":{"className:":"VectorFileSinkOperator","native:":"false"},"Statistics:":"Num rows: 33 Data size: 7348 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:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]","dataColumns:":["ctinyint:tinyint",
 "csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0","scratchColumnTypeNames:":["bigint","bigint","bigint","bigint","double","double","string","string","timestamp","timestamp","bigint","bigint"]}},"Local Work:":{"Map Reduce Local Work":{}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-4":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-4"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-4":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)","outputColumnNames:":["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_co
 l7","_col8","_col9","_col10","_col11"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col2 (type: int)","1":"_col2 (type: int)"},"OperatorId:":"HASHTABLESINK_10"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)","outputColumnNames:":["_col0","_col1","_col2","_col3","_col4",
 "_col5","_col6","_col7","_col8","_col9","_col10","_col11"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_12","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col2 (type: int)","1":"_col2 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"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"],"Statistics:":"Num rows: 33 Data size: 7348 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_13","children":{"File Output Operator":{"compressed:":"false","File Sink Vectorization:":{"className:":"VectorFileSinkOperator","native:":"false"},"Statistics:":"Num rows: 33 Data size: 7348 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"},"OperatorId:":"FS_14"}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:
 ":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0","scratchColumnTypeNames:":["bigint","bigint","bigint","bigint","double","double","string","string","timestamp","timestamp","bigint","bigint"]}},"Local Work:":{"Map Reduce Local Work":{}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_15"}}}}}}
 PREHOOK: query: select * 
 from small_alltypesorc_b c
 left outer join small_alltypesorc_b cd
@@ -337,7 +337,7 @@ from small_alltypesorc_b c
 left outer join small_alltypesorc_b hd
   on hd.ctinyint = c.ctinyint
 POSTHOOK: type: QUERY
-{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-4":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-4"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-4":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"}}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: 
 NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint)","outputColumnNames:":["_col0"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0]"},"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"out
 putColumnNames:":["_col0"],"Statistics:":"Num rows: 33 Data size: 7348 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","File Sink Vectorization:":{"className:":"VectorFileSinkOperator","native:":"false"},"Statistics:":"Num rows: 33 Data size: 7348 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:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[0]","dataColumns:":["ctinyint:tinyint"
 ,"csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Map Reduce Local Work":{}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-4":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-4"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-4":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"OperatorId:":"HASHTABLESINK_10"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Sta
 tistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint)","outputColumnNames:":["_col0"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0]"},"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_12","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Tabl
 e and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0"],"Statistics:":"Num rows: 33 Data size: 7348 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_13","children":{"File Output Operator":{"compressed:":"false","File Sink Vectorization:":{"className:":"VectorFileSinkOperator","native:":"false"},"Statistics:":"Num rows: 33 Data size: 7348 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"},"OperatorId:":"FS_14"}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNat
 ive:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[0]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Map Reduce Local Work":{}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_15"}}}}}}
 PREHOOK: query: select c.ctinyint 
 from small_alltypesorc_b c
 left outer join small_alltypesorc_b hd
@@ -780,7 +780,7 @@ left outer join small_alltypesorc_b hd
   on hd.ctinyint = c.ctinyint
 ) t1
 POSTHOOK: type: QUERY
-{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"Select Operator":{"expressions:":"cint (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: int)","1":"_col0 (type: int)"}}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE
 ","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"}}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint), cint (type: int)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 2]"},"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:"
 :{"0":"_col1 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0"],"Statistics:":"Num rows: 33 Data size: 7348 Basic stats: COMPLETE Column stats: NONE","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS 
 true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 36 Data size: 8082 Basic stats: COMPLETE Column stats: NONE","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"className:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","Not ACID UPDATE or DELETE IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys I
 S true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeColumns:":"[0, 2]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Map Reduce Local Work"
 :{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{}}}}}}
+{"PLAN VECTORIZATION":{"enabled":true,"enabledConditionsMet":["hive.vectorized.execution.enabled IS true"]},"STAGE DEPENDENCIES":{"Stage-8":{"ROOT STAGE":"TRUE"},"Stage-3":{"DEPENDENT STAGES":"Stage-8"},"Stage-0":{"DEPENDENT STAGES":"Stage-3"}},"STAGE PLANS":{"Stage-8":{"Map Reduce Local Work":{"Alias -> Map Local Tables:":{"$hdt$_1:cd":{"Fetch Operator":{"limit:":"-1"}},"$hdt$_2:hd":{"Fetch Operator":{"limit:":"-1"}}},"Alias -> Map Local Operator Tree:":{"$hdt$_1:cd":{"TableScan":{"alias:":"cd","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_2","children":{"Select Operator":{"expressions:":"cint (type: int)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_3","children":{"HashTable Sink Operator":{"keys:":{"0":"_col1 (type: int)","1":"_col0 (type: int)"},"OperatorId:":"HASHTABLESINK_26"}}}}}},"$hdt$_2:hd":{"TableScan":{"alias:":"hd","Statist
 ics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"TS_4","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint)","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_5","children":{"HashTable Sink Operator":{"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"OperatorId:":"HASHTABLESINK_24"}}}}}}}}},"Stage-3":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"c","Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","TableScan Vectorization:":{"native:":"true","projectedOutputColumns:":"[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]"},"OperatorId:":"TS_0","children":{"Select Operator":{"expressions:":"ctinyint (type: tinyint), cint (type: int)","outputColumnNames:":["_col0","_col1"],"Select Vectorization:":{"className:":"VectorSelectOperator","native:":"true","projectedOutputColumns:":"[0, 2]"},"
 Statistics:":"Num rows: 30 Data size: 6680 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"SEL_28","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col1 (type: int)","1":"_col0 (type: int)"},"Map Join Vectorization:":{"className:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"outputColumnNames:":["_col0"],"Statistics:":"Num rows: 33 Data size: 7348 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_29","children":{"Map Join Operator":{"condition map:":[{"":"Left Outer Join0 to 1"}],"keys:":{"0":"_col0 (type: tinyint)","1":"_col0 (type: tinyint)"},"Map Join Vectorization:":{"class
 Name:":"VectorMapJoinOperator","native:":"false","nativeConditionsMet:":["hive.mapjoin.optimized.hashtable IS true","hive.vectorized.execution.mapjoin.native.enabled IS true","One MapJoin Condition IS true","No nullsafe IS true","Small table vectorizes IS true","Optimized Table and Supports Key Types IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Statistics:":"Num rows: 36 Data size: 8082 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"MAPJOIN_30","children":{"Group By Operator":{"aggregations:":["count()"],"Group By Vectorization:":{"aggregators:":["VectorUDAFCountStar(*) -> bigint"],"className:":"VectorGroupByOperator","vectorOutput:":"true","native:":"false","projectedOutputColumns:":"[0]"},"mode:":"hash","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_31","children":{"Reduce Output Operator":{"sort order:":"","Reduce Sink Vectorization:":{"cla
 ssName:":"VectorReduceSinkOperator","native:":"false","nativeConditionsMet:":["hive.vectorized.execution.reducesink.new.enabled IS true","Not ACID UPDATE or DELETE IS true","No buckets IS true","No TopN IS true","No DISTINCT columns IS true","BinarySortableSerDe for keys IS true","LazyBinarySerDe for values IS true"],"nativeConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false","Uniform Hash IS false"]},"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","value expressions:":"_col0 (type: bigint)","OperatorId:":"RS_32","OutputOperators:":"[GBY_15]"}}}}}}}}}}}}],"Execution mode:":"vectorized","Map Vectorization:":{"enabled:":"true","enabledConditionsMet:":["hive.vectorized.use.vectorized.input.format IS true"],"groupByVectorOutput:":"true","inputFileFormats:":["org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"],"allNative:":"false","usesVectorUDFAdaptor:":"false","vectorized:":"true","rowBatchContext:":{"dataColumnCount:":"12","includeCol
 umns:":"[0, 2]","dataColumns:":["ctinyint:tinyint","csmallint:smallint","cint:int","cbigint:bigint","cfloat:float","cdouble:double","cstring1:string","cstring2:string","ctimestamp1:timestamp","ctimestamp2:timestamp","cboolean1:boolean","cboolean2:boolean"],"partitionColumnCount:":"0"}},"Local Work:":{"Map Reduce Local Work":{}},"Reduce Vectorization:":{"enabled:":"false","enableConditionsMet:":["hive.vectorized.execution.reduce.enabled IS true"],"enableConditionsNotMet:":["hive.execution.engine mr IN [tez, spark] IS false"]},"Reduce Operator Tree:":{"Group By Operator":{"aggregations:":["count(VALUE._col0)"],"Group By Vectorization:":{"vectorOutput:":"false","native:":"false","projectedOutputColumns:":"null"},"mode:":"mergepartial","outputColumnNames:":["_col0"],"Statistics:":"Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE","OperatorId:":"GBY_15","children":{"File Output Operator":{"compressed:":"false","Statistics:":"Num rows: 1 Data size: 8 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"},"OperatorId:":"FS_17"}}}}}},"Stage-0":{"Fetch Operator":{"limit:":"-1","Processor Tree:":{"ListSink":{"OperatorId:":"LIST_SINK_33"}}}}}}
 PREHOOK: query: select count(*) from (select c.ctinyint
 from small_alltypesorc_b c
 left outer join small_alltypesorc_b cd


[45/50] [abbrv] hive git commit: HIVE-15964: LLAP: Llap IO codepath not getting invoked due to file column id mismatch (Rajesh Balamohan, reviewed by Prasanth Jayachandran, Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-15964: LLAP: Llap IO codepath not getting invoked due to file column id mismatch (Rajesh Balamohan, reviewed by Prasanth Jayachandran, Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 539d3c6264dc8ae462e906a390dcb1d45a63422e
Parents: 53f0335
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Fri Feb 24 03:50:09 2017 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Fri Feb 24 03:50:09 2017 +0530

----------------------------------------------------------------------
 .../hive/llap/io/api/impl/LlapRecordReader.java |  11 +-
 .../test/queries/clientpositive/llap_reader.q   |  40 +++++
 .../clientpositive/llap/llap_reader.q.out       | 167 +++++++++++++++++++
 .../results/clientpositive/llap_reader.q.out    |  86 ++++++++++
 4 files changed, 301 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/539d3c62/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
index 9b1a905..d4e14a8 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
@@ -93,6 +93,8 @@ class LlapRecordReader
 
   private SchemaEvolution evolution;
 
+  private final boolean isAcidScan;
+
   public LlapRecordReader(JobConf job, FileSplit split, List<Integer> includedCols,
       String hostName, ColumnVectorProducer cvp, ExecutorService executor,
       InputFormat<?, ?> sourceInputFormat, Deserializer sourceSerDe, Reporter reporter)
@@ -139,7 +141,7 @@ class LlapRecordReader
       partitionValues = null;
     }
 
-    boolean isAcidScan = HiveConf.getBoolVar(jobConf, ConfVars.HIVE_TRANSACTIONAL_TABLE_SCAN);
+    isAcidScan = HiveConf.getBoolVar(jobConf, ConfVars.HIVE_TRANSACTIONAL_TABLE_SCAN);
     TypeDescription schema = OrcInputFormat.getDesiredRowTypeDescr(
         job, isAcidScan, Integer.MAX_VALUE);
 
@@ -169,8 +171,11 @@ class LlapRecordReader
 
   private boolean checkOrcSchemaEvolution() {
     for (int i = 0; i < columnCount; ++i) {
-      int colId = columnIds == null ? i : columnIds.get(i);
-      if (!evolution.isPPDSafeConversion(colId)) {
+      int projectedColId = columnIds == null ? i : columnIds.get(i);
+      // Adjust file column index for ORC struct.
+      // LLAP IO does not support ACID. When it supports, this would be auto adjusted.
+      int fileColId =  OrcInputFormat.getRootColumn(!isAcidScan) + projectedColId + 1;
+      if (!evolution.isPPDSafeConversion(fileColId)) {
         LlapIoImpl.LOG.warn("Unsupported schema evolution! Disabling Llap IO for {}", split);
         return false;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/539d3c62/ql/src/test/queries/clientpositive/llap_reader.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/llap_reader.q b/ql/src/test/queries/clientpositive/llap_reader.q
new file mode 100644
index 0000000..ac0624d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/llap_reader.q
@@ -0,0 +1,40 @@
+SET hive.vectorized.execution.enabled=true;
+SET hive.llap.io.enabled=true;
+SET hive.map.aggr=false;
+SET hive.exec.post.hooks=;
+
+CREATE TABLE test(f1 int, f2 int, f3 int) stored as orc;
+INSERT INTO TABLE test VALUES (1,1,1), (2,2,2), (3,3,3), (4,4,4);
+
+ALTER TABLE test CHANGE f1 f1 bigint;
+ALTER TABLE test CHANGE f2 f2 bigint;
+ALTER TABLE test CHANGE f3 f3 bigint;
+
+-- llap counters with data and meta cache
+SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecTezSummaryPrinter;
+SELECT count(f1) FROM test GROUP BY f1;
+SELECT count(f1) FROM test GROUP BY f1;
+
+SET hive.exec.post.hooks=;
+CREATE TABLE test_bigint(f1 bigint, f2 bigint, f3 bigint) stored as orc;
+INSERT OVERWRITE TABLE test_bigint select * from test;
+ALTER TABLE test_bigint CHANGE f1 f1 double;
+ALTER TABLE test_bigint CHANGE f2 f2 double;
+ALTER TABLE test_bigint CHANGE f3 f3 double;
+
+-- llap counters with meta cache alone
+SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecTezSummaryPrinter;
+select count(f1) from test_bigint group by f1;
+select count(f1) from test_bigint group by f1;
+
+
+-- Check with ACID table
+SET hive.exec.post.hooks=;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.support.concurrency=true;
+CREATE TABLE test_acid (f1 int, f2 int, val string) clustered by (val) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+INSERT INTO TABLE test_acid VALUES (1,1,'b1'), (2,2,'b2'), (3,3,'b3'), (4,4,'b4');
+
+-- should not have llap counters
+SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecTezSummaryPrinter;
+SELECT count(f1) FROM test_acid GROUP BY f1;

http://git-wip-us.apache.org/repos/asf/hive/blob/539d3c62/ql/src/test/results/clientpositive/llap/llap_reader.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/llap_reader.q.out b/ql/src/test/results/clientpositive/llap/llap_reader.q.out
new file mode 100644
index 0000000..cc556a9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/llap_reader.q.out
@@ -0,0 +1,167 @@
+PREHOOK: query: CREATE TABLE test(f1 int, f2 int, f3 int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test
+PREHOOK: query: INSERT INTO TABLE test VALUES (1,1,1), (2,2,2), (3,3,3), (4,4,4)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@test
+PREHOOK: query: ALTER TABLE test CHANGE f1 f1 bigint
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test
+PREHOOK: Output: default@test
+PREHOOK: query: ALTER TABLE test CHANGE f2 f2 bigint
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test
+PREHOOK: Output: default@test
+PREHOOK: query: ALTER TABLE test CHANGE f3 f3 bigint
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test
+PREHOOK: Output: default@test
+PREHOOK: query: SELECT count(f1) FROM test GROUP BY f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test
+#### A masked pattern was here ####
+Stage-1 FILE SYSTEM COUNTERS:
+   HDFS_BYTES_READ: 358
+   HDFS_BYTES_WRITTEN: 143
+   HDFS_READ_OPS: 6
+   HDFS_LARGE_READ_OPS: 0
+   HDFS_WRITE_OPS: 2
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 4
+   RECORDS_OUT_0: 4
+   RECORDS_OUT_INTERMEDIATE_Map_1: 4
+Stage-1 LLAP IO COUNTERS:
+   ALLOCATED_BYTES: 262144
+   ALLOCATED_USED_BYTES: 4
+   CACHE_MISS_BYTES: 7
+   METADATA_CACHE_MISS: 2
+   NUM_DECODED_BATCHES: 1
+   NUM_VECTOR_BATCHES: 1
+   ROWS_EMITTED: 4
+   SELECTED_ROWGROUPS: 1
+1
+1
+1
+1
+PREHOOK: query: SELECT count(f1) FROM test GROUP BY f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test
+#### A masked pattern was here ####
+Stage-1 FILE SYSTEM COUNTERS:
+   HDFS_BYTES_READ: 0
+   HDFS_BYTES_WRITTEN: 143
+   HDFS_READ_OPS: 2
+   HDFS_LARGE_READ_OPS: 0
+   HDFS_WRITE_OPS: 2
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 4
+   RECORDS_OUT_0: 4
+   RECORDS_OUT_INTERMEDIATE_Map_1: 4
+Stage-1 LLAP IO COUNTERS:
+   CACHE_HIT_BYTES: 7
+   CACHE_MISS_BYTES: 0
+   METADATA_CACHE_HIT: 2
+   NUM_DECODED_BATCHES: 1
+   NUM_VECTOR_BATCHES: 1
+   ROWS_EMITTED: 4
+   SELECTED_ROWGROUPS: 1
+1
+1
+1
+1
+PREHOOK: query: CREATE TABLE test_bigint(f1 bigint, f2 bigint, f3 bigint) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: INSERT OVERWRITE TABLE test_bigint select * from test
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: ALTER TABLE test_bigint CHANGE f1 f1 double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_bigint
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: ALTER TABLE test_bigint CHANGE f2 f2 double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_bigint
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: ALTER TABLE test_bigint CHANGE f3 f3 double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_bigint
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: select count(f1) from test_bigint group by f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_bigint
+#### A masked pattern was here ####
+Stage-1 FILE SYSTEM COUNTERS:
+   HDFS_BYTES_READ: 595
+   HDFS_BYTES_WRITTEN: 143
+   HDFS_READ_OPS: 6
+   HDFS_LARGE_READ_OPS: 0
+   HDFS_WRITE_OPS: 2
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 4
+   RECORDS_OUT_0: 4
+   RECORDS_OUT_INTERMEDIATE_Map_1: 4
+Stage-1 LLAP IO COUNTERS:
+   METADATA_CACHE_MISS: 1
+1
+1
+1
+1
+PREHOOK: query: select count(f1) from test_bigint group by f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_bigint
+#### A masked pattern was here ####
+Stage-1 FILE SYSTEM COUNTERS:
+   HDFS_BYTES_READ: 323
+   HDFS_BYTES_WRITTEN: 143
+   HDFS_READ_OPS: 4
+   HDFS_LARGE_READ_OPS: 0
+   HDFS_WRITE_OPS: 2
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 4
+   RECORDS_OUT_0: 4
+   RECORDS_OUT_INTERMEDIATE_Map_1: 4
+Stage-1 LLAP IO COUNTERS:
+   METADATA_CACHE_HIT: 1
+1
+1
+1
+1
+PREHOOK: query: CREATE TABLE test_acid (f1 int, f2 int, val string) clustered by (val) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_acid
+PREHOOK: query: INSERT INTO TABLE test_acid VALUES (1,1,'b1'), (2,2,'b2'), (3,3,'b3'), (4,4,'b4')
+PREHOOK: type: QUERY
+PREHOOK: Output: default@test_acid
+PREHOOK: query: SELECT count(f1) FROM test_acid GROUP BY f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_acid
+#### A masked pattern was here ####
+Stage-1 FILE SYSTEM COUNTERS:
+   HDFS_BYTES_READ: 1567
+   HDFS_BYTES_WRITTEN: 143
+   HDFS_READ_OPS: 12
+   HDFS_LARGE_READ_OPS: 0
+   HDFS_WRITE_OPS: 2
+Stage-1 HIVE COUNTERS:
+   CREATED_FILES: 1
+   DESERIALIZE_ERRORS: 0
+   RECORDS_IN_Map_1: 4
+   RECORDS_OUT_0: 4
+   RECORDS_OUT_INTERMEDIATE_Map_1: 4
+1
+1
+1
+1

http://git-wip-us.apache.org/repos/asf/hive/blob/539d3c62/ql/src/test/results/clientpositive/llap_reader.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap_reader.q.out b/ql/src/test/results/clientpositive/llap_reader.q.out
new file mode 100644
index 0000000..dcbd3aa
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap_reader.q.out
@@ -0,0 +1,86 @@
+PREHOOK: query: CREATE TABLE test(f1 int, f2 int, f3 int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test
+PREHOOK: query: INSERT INTO TABLE test VALUES (1,1,1), (2,2,2), (3,3,3), (4,4,4)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@test
+PREHOOK: query: ALTER TABLE test CHANGE f1 f1 bigint
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test
+PREHOOK: Output: default@test
+PREHOOK: query: ALTER TABLE test CHANGE f2 f2 bigint
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test
+PREHOOK: Output: default@test
+PREHOOK: query: ALTER TABLE test CHANGE f3 f3 bigint
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test
+PREHOOK: Output: default@test
+PREHOOK: query: SELECT count(f1) FROM test GROUP BY f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test
+#### A masked pattern was here ####
+1
+1
+1
+1
+PREHOOK: query: SELECT count(f1) FROM test GROUP BY f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test
+#### A masked pattern was here ####
+1
+1
+1
+1
+PREHOOK: query: CREATE TABLE test_bigint(f1 bigint, f2 bigint, f3 bigint) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: INSERT OVERWRITE TABLE test_bigint select * from test
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: ALTER TABLE test_bigint CHANGE f1 f1 double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_bigint
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: ALTER TABLE test_bigint CHANGE f2 f2 double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_bigint
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: ALTER TABLE test_bigint CHANGE f3 f3 double
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@test_bigint
+PREHOOK: Output: default@test_bigint
+PREHOOK: query: select count(f1) from test_bigint group by f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_bigint
+#### A masked pattern was here ####
+1
+1
+1
+1
+PREHOOK: query: select count(f1) from test_bigint group by f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_bigint
+#### A masked pattern was here ####
+1
+1
+1
+1
+PREHOOK: query: CREATE TABLE test_acid (f1 int, f2 int, val string) clustered by (val) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_acid
+PREHOOK: query: INSERT INTO TABLE test_acid VALUES (1,1,'b1'), (2,2,'b2'), (3,3,'b3'), (4,4,'b4')
+PREHOOK: type: QUERY
+PREHOOK: Output: default@test_acid
+PREHOOK: query: SELECT count(f1) FROM test_acid GROUP BY f1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_acid
+#### A masked pattern was here ####
+1
+1
+1
+1


[13/50] [abbrv] hive git commit: Addendum to HIVE-15957

Posted by se...@apache.org.
Addendum to HIVE-15957


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

Branch: refs/heads/hive-14535
Commit: 5c293716a37c57138da642803eee68212dd418ed
Parents: 368d916
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Fri Feb 17 14:07:38 2017 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Feb 17 14:07:38 2017 -0800

----------------------------------------------------------------------
 ql/src/test/results/clientpositive/interval_arithmetic.q.out | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5c293716/ql/src/test/results/clientpositive/interval_arithmetic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/interval_arithmetic.q.out b/ql/src/test/results/clientpositive/interval_arithmetic.q.out
index c1fc738..039d3e1 100644
--- a/ql/src/test/results/clientpositive/interval_arithmetic.q.out
+++ b/ql/src/test/results/clientpositive/interval_arithmetic.q.out
@@ -624,7 +624,7 @@ STAGE PLANS:
           alias: interval_arithmetic_1
           Statistics: Num rows: 12288 Data size: 326837 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: 2017-02-19 03:04:00.0 (type: timestamp)
+            expressions: 2017-02-20 03:04:00.0 (type: timestamp)
             outputColumnNames: _col0
             Statistics: Num rows: 12288 Data size: 491520 Basic stats: COMPLETE Column stats: COMPLETE
             Limit
@@ -640,7 +640,7 @@ POSTHOOK: query: select current_date + interval '1 2:02:00' day to second + inte
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@interval_arithmetic_1
 #### A masked pattern was here ####
-2017-02-19 03:04:00
+2017-02-20 03:04:00
 PREHOOK: query: drop table interval_arithmetic_1
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@interval_arithmetic_1