hive git commit: HIVE-20203: Arrow SerDe leaks a DirectByteBuffer (Eric Wohlstadter, reviewed by Teddy Choi)

2018-07-25 Thread tchoi
Repository: hive
Updated Branches:
  refs/heads/master 9d78fac36 -> 2820fc4c6


HIVE-20203: Arrow SerDe leaks a DirectByteBuffer (Eric Wohlstadter, reviewed by 
Teddy Choi)


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

Branch: refs/heads/master
Commit: 2820fc4c6d576bb9543bb627ab6f182f17a5c771
Parents: 9d78fac
Author: Teddy Choi 
Authored: Thu Jul 26 12:03:03 2018 +0900
Committer: Teddy Choi 
Committed: Thu Jul 26 12:03:03 2018 +0900

--
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  2 ++
 .../hadoop/hive/llap/LlapArrowRecordWriter.java | 25 
 .../hive/llap/LlapOutputFormatService.java  |  4 +---
 .../hive/llap/WritableByteChannelAdapter.java   | 13 ++
 .../hive/ql/io/arrow/ArrowWrapperWritable.java  | 19 +++
 .../hadoop/hive/ql/io/arrow/Serializer.java | 17 ++---
 6 files changed, 65 insertions(+), 15 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/2820fc4c/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 18696ad..15217e7 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2657,6 +2657,8 @@ public class HiveConf extends Configuration {
 // For Arrow SerDe
 HIVE_ARROW_ROOT_ALLOCATOR_LIMIT("hive.arrow.root.allocator.limit", 
Long.MAX_VALUE,
 "Arrow root allocator memory size limitation in bytes."),
+HIVE_ARROW_BATCH_ALLOCATOR_LIMIT("hive.arrow.batch.allocator.limit", 
10_000_000_000L,
+"Max bytes per arrow batch. This is a threshold, the memory is not 
pre-allocated."),
 HIVE_ARROW_BATCH_SIZE("hive.arrow.batch.size", 1000, "The number of rows 
sent in one Arrow batch."),
 
 // For Druid storage handler

http://git-wip-us.apache.org/repos/asf/hive/blob/2820fc4c/ql/src/java/org/apache/hadoop/hive/llap/LlapArrowRecordWriter.java
--
diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LlapArrowRecordWriter.java 
b/ql/src/java/org/apache/hadoop/hive/llap/LlapArrowRecordWriter.java
index 1b3a3eb..9ee1048 100644
--- a/ql/src/java/org/apache/hadoop/hive/llap/LlapArrowRecordWriter.java
+++ b/ql/src/java/org/apache/hadoop/hive/llap/LlapArrowRecordWriter.java
@@ -20,11 +20,12 @@ package org.apache.hadoop.hive.llap;
 
 import java.io.IOException;
 
+import org.apache.arrow.memory.BufferAllocator;
 import org.apache.arrow.vector.VectorSchemaRoot;
 import org.apache.arrow.vector.ipc.ArrowStreamWriter;
 import org.apache.hadoop.hive.ql.io.arrow.ArrowWrapperWritable;
+import org.apache.arrow.vector.complex.NullableMapVector;
 import org.apache.hadoop.io.Writable;
-import java.nio.channels.WritableByteChannel;
 import org.apache.hadoop.mapred.RecordWriter;
 import org.apache.hadoop.mapred.Reporter;
 import org.slf4j.Logger;
@@ -47,15 +48,28 @@ public class LlapArrowRecordWriter
   public static final Logger LOG = 
LoggerFactory.getLogger(LlapArrowRecordWriter.class);
 
   ArrowStreamWriter arrowStreamWriter;
-  WritableByteChannel out;
+  WritableByteChannelAdapter out;
+  BufferAllocator allocator;
+  NullableMapVector rootVector;
 
-  public LlapArrowRecordWriter(WritableByteChannel out) {
+  public LlapArrowRecordWriter(WritableByteChannelAdapter out) {
 this.out = out;
   }
 
   @Override
   public void close(Reporter reporter) throws IOException {
-arrowStreamWriter.close();
+try {
+  arrowStreamWriter.close();
+} finally {
+  rootVector.close();
+  //bytesLeaked should always be 0
+  long bytesLeaked = allocator.getAllocatedMemory();
+  if(bytesLeaked != 0) {
+LOG.error("Arrow memory leaked bytes: {}", bytesLeaked);
+throw new IllegalStateException("Arrow memory leaked bytes:" + 
bytesLeaked);
+  }
+  allocator.close();
+}
   }
 
   @Override
@@ -64,6 +78,9 @@ public class LlapArrowRecordWriter
 if (arrowStreamWriter == null) {
   VectorSchemaRoot vectorSchemaRoot = 
arrowWrapperWritable.getVectorSchemaRoot();
   arrowStreamWriter = new ArrowStreamWriter(vectorSchemaRoot, null, out);
+  allocator = arrowWrapperWritable.getAllocator();
+  this.out.setAllocator(allocator);
+  rootVector = arrowWrapperWritable.getRootVector();
 }
 arrowStreamWriter.writeBatch();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/2820fc4c/ql/src/java/org/apache/hadoop/hive/llap/LlapOutputFormatService.java
---

hive git commit: HIVE-20015: Populate ArrayList with Constructor (Daniel Voros, reviewed by Lazlo Bodor, Zoltan Haindrich)

2018-07-25 Thread ngangam
Repository: hive
Updated Branches:
  refs/heads/master 14bb84088 -> 9d78fac36


HIVE-20015: Populate ArrayList with Constructor (Daniel Voros, reviewed by 
Lazlo Bodor, Zoltan Haindrich)


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

Branch: refs/heads/master
Commit: 9d78fac365418dd781ead48dfc69f2143d3e7eb2
Parents: 14bb840
Author: Naveen Gangam 
Authored: Wed Jul 25 18:26:25 2018 -0400
Committer: Naveen Gangam 
Committed: Wed Jul 25 18:26:25 2018 -0400

--
 ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java | 5 +
 1 file changed, 1 insertion(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/9d78fac3/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
--
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
index 5bf0625..a0bd649 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
@@ -596,12 +596,9 @@ public class MapWork extends BaseWork {
   }
 
   public ArrayList getPaths() {
-ArrayList ret=new ArrayList<>();
-ret.addAll(pathToAliases.keySet());
-return ret;
+return new ArrayList(pathToAliases.keySet());
   }
 
-
   public ArrayList getPartitionDescs() {
 return new ArrayList(aliasToPartnInfo.values());
   }



hive git commit: HIVE-19986: Add logging of runtime statistics indicating when Hdfs Erasure Coding is used by MR (Andrew Sherman, reviewed by Sahil Takiar)

2018-07-25 Thread stakiar
Repository: hive
Updated Branches:
  refs/heads/master 68bdf9eb4 -> 14bb84088


HIVE-19986: Add logging of runtime statistics indicating when Hdfs Erasure 
Coding is used by MR (Andrew Sherman, reviewed by Sahil Takiar)


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

Branch: refs/heads/master
Commit: 14bb84088f65467d2ea0cc828a48cd33e3ec
Parents: 68bdf9e
Author: Andrew Sherman 
Authored: Wed Jul 25 16:23:51 2018 -0500
Committer: Sahil Takiar 
Committed: Wed Jul 25 16:44:58 2018 -0500

--
 .../jdbc/TestJdbcWithMiniHS2ErasureCoding.java  | 55 
 .../org/apache/hadoop/hive/ql/MapRedStats.java  | 43 ++-
 .../hive/ql/exec/mr/HadoopJobExecHelper.java|  2 +-
 .../hive/ql/processors/ErasureProcessor.java| 10 +++-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java | 17 ++
 .../apache/hadoop/hive/shims/HadoopShims.java   | 10 
 6 files changed, 134 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/14bb8408/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java
--
diff --git 
a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java
 
b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java
index b0a0145..efb3759 100644
--- 
a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java
+++ 
b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2ErasureCoding.java
@@ -19,6 +19,8 @@
 package org.apache.hive.jdbc;
 
 import java.io.IOException;
+import java.io.StringWriter;
+import java.io.Writer;
 import java.nio.file.Paths;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -31,11 +33,17 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.processors.ErasureProcessor;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.HadoopShims.HdfsErasureCodingShim;
 import org.apache.hadoop.hive.shims.HadoopShims.MiniDFSShim;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.appender.WriterAppender;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.layout.PatternLayout;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -174,6 +182,53 @@ public class TestJdbcWithMiniHS2ErasureCoding {
   }
 
   /**
+   * Test MR stats.
+   */
+  @Test
+  public void testMapRedStats() throws Exception {
+// Do log4j magic to save log output
+StringWriter writer = new StringWriter();
+Appender appender = addAppender(writer, "testMapRedStats");
+try (Statement stmt = hs2Conn.createStatement()) {
+  String table = "mapredstats";
+  stmt.execute("set hive.execution.engine=mr");
+  stmt.execute(" CREATE TABLE " + table + " (a int) STORED AS PARQUET");
+  stmt.execute("INSERT INTO TABLE " + table + " VALUES (3)");
+  try (ResultSet rs = stmt.executeQuery("select a from " + table + " order 
by a")) {
+while (rs.next()) {
+  int val = rs.getInt(1);
+  assertEquals(3, val);
+}
+  }
+}
+String output = writer.toString();
+// check for standard stats
+assertTrue(output.contains("HDFS Read:"));
+assertTrue(output.contains("HDFS Write:"));
+
+// check for erasure coding stat
+HadoopShims.HdfsErasureCodingShim erasureShim = 
ErasureProcessor.getErasureShim(conf);
+if (erasureShim.isMapReduceStatAvailable()) {
+  assertTrue(output.contains("HDFS EC Read:"));
+}
+  }
+
+  /**
+   * Add an appender to log4j.
+   * 
http://logging.apache.org/log4j/2.x/manual/customconfig.html#AddingToCurrent
+   */
+  private Appender addAppender(final Writer writer, final String writerName) {
+final LoggerContext context = LoggerContext.getContext(false);
+final Configuration config = context.getConfiguration();
+final PatternLayout layout = PatternLayout.createDefaultLayout(config);
+final Appender appender =
+WriterAppender.createAppender(layout, null, writer, writerName, false, 
true);
+appender.start();
+config.getRootLogger().addAppender(appender, null, null);
+return appender;
+  }
+
+  /**
* Add a Erasure Coding Policy to

hive git commit: HIVE-19441 : Use LLAP test Driver for Druid test suites (Slim Bouguerra via Ashutosh Chauhan)

2018-07-25 Thread hashutosh
Repository: hive
Updated Branches:
  refs/heads/master a15d75b47 -> 68bdf9eb4


HIVE-19441 : Use LLAP test Driver for Druid test suites (Slim Bouguerra via 
Ashutosh Chauhan)

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

Branch: refs/heads/master
Commit: 68bdf9eb4e7ef245b8eacc46ddbf94beab86be44
Parents: a15d75b
Author: Slim Bouguerra 
Authored: Wed Jul 25 13:32:22 2018 -0700
Committer: Ashutosh Chauhan 
Committed: Wed Jul 25 13:32:22 2018 -0700

--
 .../hive/cli/MiniDruidLlapLocalCliDriver.java   | 46 
 .../hadoop/hive/cli/TestMiniDruidCliDriver.java |  3 +-
 .../test/resources/testconfiguration.properties |  2 +
 .../hadoop/hive/cli/control/CliConfigs.java | 23 ++
 .../org/apache/hadoop/hive/ql/QTestUtil.java| 30 ++---
 .../clientpositive/druid/druid_basic2.q.out | 18 ++--
 .../druid/druidmini_dynamic_partition.q.out |  9 
 .../druid/druidmini_expressions.q.out   | 27 
 .../clientpositive/druid/druidmini_joins.q.out  |  8 +++-
 .../clientpositive/druid/druidmini_mv.q.out | 18 
 .../clientpositive/druid/druidmini_test1.q.out  | 18 
 11 files changed, 190 insertions(+), 12 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/68bdf9eb/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/MiniDruidLlapLocalCliDriver.java
--
diff --git 
a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/MiniDruidLlapLocalCliDriver.java
 
b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/MiniDruidLlapLocalCliDriver.java
new file mode 100644
index 000..fa74e49
--- /dev/null
+++ 
b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/MiniDruidLlapLocalCliDriver.java
@@ -0,0 +1,46 @@
+package org.apache.hadoop.hive.cli;
+
+import org.apache.hadoop.hive.cli.control.CliAdapter;
+import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.util.List;
+
+/**
+ * This is used only for dev debugging if needed
+ */
+@RunWith(Parameterized.class)
+public class MiniDruidLlapLocalCliDriver {
+  static CliAdapter adapter = new 
CliConfigs.MiniDruidLlapLocalCliConfig().getCliAdapter();
+
+  @Parameterized.Parameters(name = "{0}")
+  public static List getParameters() throws Exception {
+return adapter.getParameters();
+  }
+
+  @ClassRule
+  public static TestRule cliClassRule = adapter.buildClassRule();
+
+  @Rule
+  public TestRule cliTestRule = adapter.buildTestRule();
+
+  private String name;
+  private File qfile;
+
+  public MiniDruidLlapLocalCliDriver(String name, File qfile) {
+this.name = name;
+this.qfile = qfile;
+  }
+
+  @Test
+  public void testCliDriver() throws Exception {
+adapter.runTest(name, qfile);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/68bdf9eb/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidCliDriver.java
--
diff --git 
a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidCliDriver.java
 
b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidCliDriver.java
index fa75d65..6269547 100644
--- 
a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidCliDriver.java
+++ 
b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidCliDriver.java
@@ -31,7 +31,8 @@ import java.io.File;
 import java.util.List;
 
 @RunWith(Parameterized.class)
-public class TestMiniDruidCliDriver {
+public class TestMiniDruidCliDriver
+{
 
   static CliAdapter adapter = new 
CliConfigs.MiniDruidCliConfig().getCliAdapter();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/68bdf9eb/itests/src/test/resources/testconfiguration.properties
--
diff --git a/itests/src/test/resources/testconfiguration.properties 
b/itests/src/test/resources/testconfiguration.properties
index b5ae390..50bfe6a 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1712,6 +1712,8 @@ druid.query.files=druidmini_test1.q,\
   druidmini_floorTime.q, \
   druidmini_masking.q
 
+druid.llap.local.query.files=druidmini_noop.q
+
 druid.kafka.query.files=druidkafkamini_basic.q
 
 # tests to be run by TestErasureCodingHDFSCliDriver and TestCliDriver

http://g

hive git commit: HIVE-19766: Show the number of rows inserted when execution engine is Spark (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)

2018-07-25 Thread stakiar
Repository: hive
Updated Branches:
  refs/heads/master 758ff4490 -> a15d75b47


HIVE-19766: Show the number of rows inserted when execution engine is Spark 
(Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)


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

Branch: refs/heads/master
Commit: a15d75b47af7acbe567d30c865e05f21c5ca7229
Parents: 758ff44
Author: Bharathkrishna Guruvayoor Murali 
Authored: Wed Jul 25 14:11:47 2018 -0500
Committer: Sahil Takiar 
Committed: Wed Jul 25 14:11:47 2018 -0500

--
 .../apache/hadoop/hive/ql/exec/spark/SparkTask.java   | 14 ++
 1 file changed, 14 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/a15d75b4/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
--
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
index ad5049a..9277510 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
@@ -38,6 +38,8 @@ import 
org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames;
 import org.apache.hadoop.hive.ql.exec.spark.status.impl.SparkMetricsUtils;
 
 import org.apache.hadoop.hive.ql.exec.spark.status.SparkStage;
+import org.apache.hive.spark.counter.SparkCounter;
+import org.apache.hive.spark.counter.SparkCounters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -163,6 +165,17 @@ public class SparkTask extends Task {
 
   if (rc == 0) {
 sparkStatistics = sparkJobStatus.getSparkStatistics();
+if (SessionState.get() != null) {
+  //Set the number of rows written in case of insert queries, to print 
in the client(beeline).
+  SparkCounters counters = sparkJobStatus.getCounter();
+  if (counters != null) {
+SparkCounter counter = counters.getCounter(HiveConf.getVar(conf, 
HiveConf.ConfVars.HIVECOUNTERGROUP),
+FileSinkOperator.TOTAL_TABLE_ROWS_WRITTEN);
+if (counter != null) {
+  queryState.setNumModifiedRows(counter.getValue());
+}
+  }
+}
 printConsoleMetrics();
 printExcessiveGCWarning();
 if (LOG.isInfoEnabled() && sparkStatistics != null) {
@@ -500,6 +513,7 @@ public class SparkTask extends Task {
 List hiveCounters = new LinkedList();
 counters.put(groupName, hiveCounters);
 hiveCounters.add(Operator.HIVE_COUNTER_CREATED_FILES);
+hiveCounters.add(FileSinkOperator.TOTAL_TABLE_ROWS_WRITTEN);
 // MapOperator is out of SparkWork, SparkMapRecordHandler use it to bridge
 // Spark transformation and Hive operators in SparkWork.
 for (MapOperator.Counter counter : MapOperator.Counter.values()) {



[19/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index 000,c2bbba5..7b32c08
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@@ -1,0 -1,1686 +1,1688 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  * 
+  * http://www.apache.org/licenses/LICENSE-2.0
+  * 
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hive.metastore.conf;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.metastore.DefaultStorageSchemaReader;
+ import org.apache.hadoop.hive.metastore.HiveAlterHandler;
+ import 
org.apache.hadoop.hive.metastore.MaterializationsRebuildLockCleanerTask;
+ import org.apache.hadoop.hive.metastore.MetastoreTaskThread;
+ import org.apache.hadoop.hive.metastore.RuntimeStatsCleanerTask;
+ import org.apache.hadoop.hive.metastore.events.EventCleanerTask;
+ import 
org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager;
+ import org.apache.hadoop.hive.metastore.txn.AcidCompactionHistoryService;
+ import org.apache.hadoop.hive.metastore.txn.AcidHouseKeeperService;
+ import org.apache.hadoop.hive.metastore.txn.AcidOpenTxnsCounterService;
+ import org.apache.hadoop.hive.metastore.txn.AcidWriteSetService;
+ import org.apache.hadoop.hive.metastore.utils.StringUtils;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import java.io.File;
+ import java.io.IOException;
+ import java.net.URI;
+ import java.net.URL;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.regex.Matcher;
+ import java.util.regex.Pattern;
+ 
+ /**
+  * A set of definitions of config values used by the Metastore.  One of the 
key aims of this
+  * class is to provide backwards compatibility with existing Hive 
configuration keys while
+  * allowing the metastore to have its own, Hive independent keys.   For this 
reason access to the
+  * underlying Configuration object should always be done via the static 
methods provided here
+  * rather than directly via {@link Configuration#get(String)} and
+  * {@link Configuration#set(String, String)}.  All the methods of this class 
will handle checking
+  * both the MetastoreConf key and the Hive key.  The algorithm is, on reads, 
to check first the
+  * MetastoreConf key, then the Hive key, then return the default if neither 
are set.  On write
+  * the Metastore key only is set.
+  *
+  * This class does not extend Configuration.  Rather it provides static 
methods for operating on
+  * a Configuration object.  This allows it to work on HiveConf objects, which 
otherwise would not
+  * be the case.
+  */
+ public class MetastoreConf {
+ 
+   private static final Logger LOG = 
LoggerFactory.getLogger(MetastoreConf.class);
+   private static final Pattern TIME_UNIT_SUFFIX = 
Pattern.compile("([0-9]+)([a-zA-Z]+)");
+ 
+   private static final Map metaConfs = new HashMap<>();
+   private static URL hiveDefaultURL = null;
+   private static URL hiveSiteURL = null;
+   private static URL hiveMetastoreSiteURL = null;
+   private static URL metastoreSiteURL = null;
+   private static AtomicBoolean beenDumped = new AtomicBoolean();
+ 
+   private static Map keyToVars;
+ 
+   @VisibleForTesting
+   static final String TEST_ENV_WORKAROUND = 
"metastore.testing.env.workaround.dont.ever.set.this.";
+ 
+   public static enum StatsUpdateMode {
+ NONE, EXISTING, ALL
+   }
+ 
+   private static class TimeValue {
+ final long val;
+ final TimeUnit unit;
+ 
+ private TimeValue(long val, TimeUnit unit) {
+   this.val = val;
+  

[20/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 000,8ff056f..9bee0db
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@@ -1,0 -1,2532 +1,2532 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hive.metastore.cache;
+ 
+ 
+ import java.nio.ByteBuffer;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.EmptyStackException;
+ import java.util.HashMap;
+ import java.util.LinkedList;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Stack;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.ScheduledExecutorService;
+ import java.util.concurrent.ThreadFactory;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.regex.Matcher;
+ import java.util.regex.Pattern;
+ 
+ import org.apache.hadoop.conf.Configurable;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.common.DatabaseName;
+ import org.apache.hadoop.hive.common.StatsSetupConst;
+ import org.apache.hadoop.hive.common.TableName;
+ import org.apache.hadoop.hive.metastore.Deadline;
+ import org.apache.hadoop.hive.metastore.FileMetadataHandler;
+ import org.apache.hadoop.hive.metastore.ObjectStore;
+ import org.apache.hadoop.hive.metastore.PartFilterExprUtil;
+ import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
+ import org.apache.hadoop.hive.metastore.RawStore;
+ import org.apache.hadoop.hive.metastore.TableType;
+ import org.apache.hadoop.hive.metastore.Warehouse;
 -import org.apache.hadoop.hive.metastore.api.AggrStats;
 -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 -import org.apache.hadoop.hive.metastore.api.Catalog;
 -import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 -import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 -import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 -import org.apache.hadoop.hive.metastore.api.Database;
 -import org.apache.hadoop.hive.metastore.api.FieldSchema;
 -import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 -import org.apache.hadoop.hive.metastore.api.Function;
 -import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 -import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 -import org.apache.hadoop.hive.metastore.api.ISchema;
 -import org.apache.hadoop.hive.metastore.api.ISchemaName;
 -import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 -import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 -import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
 -import org.apache.hadoop.hive.metastore.api.MetaException;
 -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 -import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventRequest;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventsCountResponse;
 -import org.apache.hadoop.hive.metastore.api.Partition;
 -import org.apache.hadoop.hive.metastore.api.PartitionEventType;
 -import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
 -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 -import org.apache.hadoop.hive.metastore.api.PrincipalType;
 -import org.apac

[06/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
--
diff --cc 
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
index 000,abbcda3..c5977b2
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/InjectableBehaviourObjectStore.java
@@@ -1,0 -1,211 +1,218 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import java.util.List;
++
+ import org.apache.hadoop.hive.metastore.api.Function;
+ import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventRequest;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+ import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ 
+ import static org.junit.Assert.assertEquals;
+ 
+ 
+ /**
+  * A wrapper around {@link ObjectStore} that allows us to inject custom 
behaviour
+  * on to some of the methods for testing.
+  */
+ public class InjectableBehaviourObjectStore extends ObjectStore {
+   public InjectableBehaviourObjectStore() {
+ super();
+   }
+ 
+   /**
+* A utility class that allows people injecting behaviour to determine if 
their injections occurred.
+*/
+   public static abstract class BehaviourInjection
+   implements com.google.common.base.Function{
+ protected boolean injectionPathCalled = false;
+ protected boolean nonInjectedPathCalled = false;
+ 
+ public void assertInjectionsPerformed(
+ boolean expectedInjectionCalled, boolean 
expectedNonInjectedPathCalled){
+   assertEquals(expectedInjectionCalled, injectionPathCalled);
+   assertEquals(expectedNonInjectedPathCalled, nonInjectedPathCalled);
+ }
+   }
+ 
+   /**
+* A utility class to pass the arguments of the caller to the stub method.
+*/
+   public class CallerArguments {
+ public String dbName;
+ public String tblName;
+ public String funcName;
+ public String constraintTblName;
+ 
+ public CallerArguments(String dbName) {
+   this.dbName = dbName;
+ }
+   }
+ 
+   private static com.google.common.base.Function 
getTableModifier =
+   com.google.common.base.Functions.identity();
+   private static com.google.common.base.Function 
getPartitionModifier =
+   com.google.common.base.Functions.identity();
+   private static com.google.common.base.Function, List> 
listPartitionNamesModifier =
+   com.google.common.base.Functions.identity();
+   private static com.google.common.base.Function
+   getNextNotificationModifier = 
com.google.common.base.Functions.identity();
+ 
+   private static com.google.common.base.Function 
callerVerifier = null;
+ 
+   // Methods to set/reset getTable modifier
+   public static void 
setGetTableBehaviour(com.google.common.base.Function modifier){
+ getTableModifier = (modifier == null) ? 
com.google.common.base.Functions.identity() : modifier;
+   }
+ 
+   public static void resetGetTableBehaviour(){
+ setGetTableBehaviour(null);
+   }
+ 
+   // Methods to set/reset getPartition modifier
+   public static void 
setGetPartitionBehaviour(com.google.common.base.Function 
modifier){
+ getPartitionModifier = (modifier == null) ? 
com.google.common.base.Functions.identity() : modifier;
+   }
+ 
+   public static void resetGetPartitionBehaviour(){
+ setGetPartitionBehaviour(null);
+   }
+ 
+   // Methods to set/reset listPartitionNames modifier
+   public static void 
setListPartit

[29/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719


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

Branch: refs/heads/master
Commit: 651e7950977dd4e63da42648c38b03c3bf097e7f
Parents: f0a2fff 851c8ab
Author: sergey 
Authored: Thu Jul 19 14:44:10 2018 -0700
Committer: sergey 
Committed: Thu Jul 19 14:44:10 2018 -0700

--
 .gitignore  | 1 +
 RELEASE_NOTES.txt   | 8 +
 .../org/apache/hive/beeline/BeeLineOpts.java|11 +-
 .../apache/hive/beeline/cli/TestHiveCli.java| 2 -
 .../org/apache/hadoop/hive/conf/HiveConf.java   | 2 +
 .../HiveHBaseTableSnapshotInputFormat.java  | 4 +-
 hcatalog/core/pom.xml   | 7 +
 hcatalog/webhcat/java-client/pom.xml| 7 +
 hcatalog/webhcat/svr/pom.xml| 7 +
 itests/hcatalog-unit/pom.xml| 6 +
 itests/hive-blobstore/pom.xml   |13 +
 .../insert_overwrite_directory.q.out| 2 +
 .../write_final_output_blobstore.q.out  | 8 +
 itests/hive-minikdc/pom.xml |13 +
 itests/hive-unit-hadoop2/pom.xml| 6 +
 itests/hive-unit/pom.xml| 8 +-
 itests/qtest-accumulo/pom.xml   |13 +
 itests/qtest-spark/pom.xml  |13 +
 itests/qtest/pom.xml|13 +
 .../test/resources/testconfiguration.properties | 7 +-
 itests/util/pom.xml | 6 +
 llap-server/pom.xml | 7 +
 metastore/pom.xml   | 5 +
 packaging/src/main/assembly/bin.xml | 2 +-
 packaging/src/main/assembly/src.xml | 1 +
 ql/pom.xml  | 8 +
 .../hadoop/hive/ql/plan/api/OperatorType.java   | 5 +-
 ...eColumnArithmeticIntervalYearMonthColumn.txt | 3 +-
 ...YearMonthColumnArithmeticTimestampColumn.txt | 4 +-
 .../java/org/apache/hadoop/hive/ql/Context.java |10 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 5 +-
 .../org/apache/hadoop/hive/ql/QueryPlan.java| 9 +
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |13 +-
 .../hadoop/hive/ql/exec/KeyWrapperFactory.java  | 2 +-
 .../hadoop/hive/ql/exec/OperatorFactory.java| 4 +
 .../hadoop/hive/ql/exec/TableScanOperator.java  | 7 +
 .../hadoop/hive/ql/exec/TopNKeyOperator.java|   214 +
 .../hadoop/hive/ql/exec/repl/ReplLoadTask.java  | 6 +-
 .../IncrementalLoadTasksBuilder.java|10 +-
 .../hive/ql/exec/vector/VectorAssignRow.java|20 +-
 .../ql/exec/vector/VectorTopNKeyOperator.java   |   304 +
 .../apache/hadoop/hive/ql/hooks/ATSHook.java| 3 +-
 .../hive/ql/hooks/HiveProtoLoggingHook.java | 3 +-
 .../metadata/HiveMaterializedViewsRegistry.java |11 +-
 .../hive/ql/optimizer/TopNKeyProcessor.java |   109 +
 .../ql/optimizer/calcite/RelOptHiveTable.java   |73 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |37 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java|86 +-
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  | 3 +-
 .../hadoop/hive/ql/parse/TezCompiler.java   |27 +
 .../apache/hadoop/hive/ql/plan/ExplainWork.java |14 +-
 .../apache/hadoop/hive/ql/plan/TopNKeyDesc.java |   139 +
 .../hadoop/hive/ql/plan/VectorTopNKeyDesc.java  |39 +
 .../hadoop/hive/ql/txn/compactor/Worker.java| 1 +
 .../hive/ql/udf/generic/GenericUDAFCount.java   | 6 +-
 .../hadoop/hive/ql/exec/TestExplainTask.java| 2 +-
 .../ql/exec/vector/VectorRandomRowSource.java   |35 +-
 .../vector/aggregation/AggregationBase.java |22 +-
 .../aggregation/TestVectorAggregation.java  |   133 +-
 .../expressions/TestVectorArithmetic.java   |   143 +-
 .../expressions/TestVectorCastStatement.java| 2 +
 .../expressions/TestVectorDateAddSub.java   | 4 +
 .../vector/expressions/TestVectorDateDiff.java  | 4 +
 .../expressions/TestVectorIfStatement.java  | 2 +
 .../vector/expressions/TestVectorNegative.java  | 2 +
 .../expressions/TestVectorStringConcat.java | 2 +
 .../expressions/TestVectorStringUnary.java  | 2 +
 .../vector/expressions/TestVectorSubStr.java| 2 +
 .../expressions/TestVectorTimestampExtract.java | 3 +
 .../parse/TestUpdateDeleteSemanticAnalyzer.java | 2 +-
 ql/src/test/queries/clientpositive/bucket7.q|12 +
 ql/src/test/queries/clientpositive/topnkey.q|31 +
 .../queries/clientpositive/vector_topnkey.q |30 +
 .../results/clientpositive/acid_nullscan.q.out  | 

[03/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
--
diff --cc 
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
index 000,62ed380..e4854f9
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
@@@ -1,0 -1,1075 +1,1075 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hive.metastore.cache;
+ 
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.HashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.ExecutorService;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.ThreadFactory;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
+ import org.apache.hadoop.hive.metastore.HiveMetaStore;
+ import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+ import org.apache.hadoop.hive.metastore.ObjectStore;
+ import org.apache.hadoop.hive.metastore.TableType;
+ import org.apache.hadoop.hive.metastore.Warehouse;
+ import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+ import org.apache.hadoop.hive.metastore.api.AggrStats;
+ import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.PrincipalType;
+ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+ import 
org.apache.hadoop.hive.metastore.columnstats.cache.LongColumnStatsDataInspector;
+ import 
org.apache.hadoop.hive.metastore.columnstats.cache.StringColumnStatsDataInspector;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+ import org.junit.Assert;
+ import org.junit.Before;
+ import org.junit.Test;
+ import org.junit.experimental.categories.Category;
+ 
+ import jline.internal.Log;
+ 
+ import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+ 
+ @Category(MetastoreCheckinTest.class)
+ public class TestCachedStore {
+ 
+   private ObjectStore objectStore;
+   private CachedStore cachedStore;
+   private SharedCache sharedCache;
+   private Configuration conf;
+ 
+   @Before
+   public void setUp() throws Exception {
+ conf = MetastoreConf.newMetastoreConf();
+ MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.HIVE_IN_TEST, true);
+ // Disable memory estimation for this test class
+ MetastoreConf.setVar(conf, 
MetastoreConf.ConfVars.CACHED_RAW_STORE_MAX_CACHE_MEMORY, "-1Kb");
+ MetaStoreTestUtils.setConfForStandloneMode(conf);
+ objectStore = new ObjectStore();
+ objectStore.setConf(conf);
+ cachedStore = new CachedStore();
+ cachedStore.setConfForTest(conf);
+ // Stop the CachedStore cache update service. We'll start it explicitly 
to control the test
+ CachedStore.stopCacheUpdateService(1);
+ sharedCache = new SharedCache();
+ sharedCache.getDatabaseCache().clear();
+ sharedCache.getTableCache().clear();
+ sharedCache.getSdCache().clear();
+ 
+ // Create the 'hive' catalog
+ HiveMetaSto

[24/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 000,9661beb..70be8d8
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@@ -1,0 -1,3699 +1,3757 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ 
+ import java.io.IOException;
+ import java.nio.ByteBuffer;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Map.Entry;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.common.ValidTxnList;
+ import org.apache.hadoop.hive.common.ValidWriteIdList;
+ import org.apache.hadoop.hive.common.classification.RetrySemantics;
+ import org.apache.hadoop.hive.metastore.annotation.NoReconnect;
+ import org.apache.hadoop.hive.metastore.api.AggrStats;
+ import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+ import org.apache.hadoop.hive.metastore.api.CheckConstraintsRequest;
+ import org.apache.hadoop.hive.metastore.api.Catalog;
+ import org.apache.hadoop.hive.metastore.api.CmRecycleRequest;
+ import org.apache.hadoop.hive.metastore.api.CmRecycleResponse;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+ import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
+ import org.apache.hadoop.hive.metastore.api.CompactionResponse;
+ import org.apache.hadoop.hive.metastore.api.CompactionType;
+ import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
+ import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+ import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+ import org.apache.hadoop.hive.metastore.api.DataOperationType;
+ import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.FindSchemasByColsResp;
+ import org.apache.hadoop.hive.metastore.api.FindSchemasByColsRqst;
+ import org.apache.hadoop.hive.metastore.api.FireEventRequest;
+ import org.apache.hadoop.hive.metastore.api.FireEventResponse;
+ import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
+ import org.apache.hadoop.hive.metastore.api.Function;
+ import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
+ import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
+ import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
+ import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse;
+ import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
+ import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+ import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
+ import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+ import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+ import org.apache.hadoop.hive.metastore.api.ISchema;
+ import org.apache.hadoop.hive.metastore.api.InvalidInputException;
+ import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+ import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
+ import org.apache.hadoop.hive.metastore.api.LockRequest;
+ import org.apache.hadoop.hive.metastore.api.LockResponse;
+ import org.apache.hadoop.hive.metastore.api.Materialization;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apach

[44/50] [abbrv] hive git commit: HIVE-19532 : fix tests for master-txnstats branch - fix build (Sergey Shelukhin)

2018-07-25 Thread sershe
HIVE-19532 : fix tests for master-txnstats branch - fix build (Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: e8d7cdcc372e14f8a0a664911b5ae6934201e30b
Parents: b17a347
Author: sergey 
Authored: Sun Jul 22 21:20:46 2018 -0700
Committer: sergey 
Committed: Sun Jul 22 21:20:46 2018 -0700

--
 .../hcatalog/listener/DummyRawStoreFailEvent.java   | 16 
 .../hadoop/hive/ql/stats/StatsUpdaterThread.java|  4 ++--
 2 files changed, 10 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/e8d7cdcc/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
--
diff --git 
a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
 
b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 1c105d1..be40395 100644
--- 
a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ 
b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -321,10 +321,10 @@ public class DummyRawStoreFailEvent implements RawStore, 
Configurable {
   }
 
   @Override
-  public void alterTable(String catName, String dbName, String name, Table 
newTable, String queryValidWriteIds)
+  public Table alterTable(String catName, String dbName, String name, Table 
newTable, String queryValidWriteIds)
   throws InvalidObjectException, MetaException {
 if (shouldEventSucceed) {
-  objectStore.alterTable(catName, dbName, name, newTable, 
queryValidWriteIds);
+  return objectStore.alterTable(catName, dbName, name, newTable, 
queryValidWriteIds);
 } else {
   throw new RuntimeException("Event failed.");
 }
@@ -385,22 +385,22 @@ public class DummyRawStoreFailEvent implements RawStore, 
Configurable {
   }
 
   @Override
-  public void alterPartition(String catName, String dbName, String tblName, 
List partVals,
+  public Partition alterPartition(String catName, String dbName, String 
tblName, List partVals,
  Partition newPart, String queryValidWriteIds) 
throws InvalidObjectException, MetaException {
 if (shouldEventSucceed) {
-  objectStore.alterPartition(catName, dbName, tblName, partVals, newPart, 
queryValidWriteIds);
+  return objectStore.alterPartition(catName, dbName, tblName, partVals, 
newPart, queryValidWriteIds);
 } else {
   throw new RuntimeException("Event failed.");
 }
   }
 
   @Override
-  public void alterPartitions(String catName, String dbName, String tblName,
+  public List alterPartitions(String catName, String dbName, String 
tblName,
   List> partValsList, List 
newParts,
   long writeId, String queryValidWriteIds)
   throws InvalidObjectException, MetaException {
 if (shouldEventSucceed) {
-  objectStore.alterPartitions(catName, dbName, tblName, partValsList, 
newParts, writeId, queryValidWriteIds);
+  return objectStore.alterPartitions(catName, dbName, tblName, 
partValsList, newParts, writeId, queryValidWriteIds);
 } else {
   throw new RuntimeException("Event failed.");
 }
@@ -736,13 +736,13 @@ public class DummyRawStoreFailEvent implements RawStore, 
Configurable {
   }
 
   @Override
-  public boolean updateTableColumnStatistics(ColumnStatistics statsObj, String 
validWriteIds, long writeId)
+  public Map updateTableColumnStatistics(ColumnStatistics 
statsObj, String validWriteIds, long writeId)
   throws NoSuchObjectException, MetaException, InvalidObjectException, 
InvalidInputException {
 return objectStore.updateTableColumnStatistics(statsObj, validWriteIds, 
writeId);
   }
 
   @Override
-  public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj,
+  public Map updatePartitionColumnStatistics(ColumnStatistics 
statsObj,
   List partVals, String validWriteIds, long writeId)
   throws NoSuchObjectException, MetaException, InvalidObjectException, 
InvalidInputException {
 return objectStore.updatePartitionColumnStatistics(statsObj, partVals, 
validWriteIds, writeId);

http://git-wip-us.apache.org/repos/asf/hive/blob/e8d7cdcc/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java 
b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUpdaterThread.java
in

[07/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
--
diff --cc 
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index 000,d91f737..bc04e06
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@@ -1,0 -1,3424 +1,3546 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import static 
org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+ import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
++import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependCatalogToDbName;
+ 
+ import java.io.IOException;
+ import java.lang.reflect.Constructor;
+ import java.lang.reflect.InvocationHandler;
+ import java.lang.reflect.InvocationTargetException;
+ import java.lang.reflect.Method;
+ import java.lang.reflect.Proxy;
+ import java.net.InetAddress;
+ import java.net.URI;
+ import java.net.UnknownHostException;
+ import java.nio.ByteBuffer;
+ import java.security.PrivilegedExceptionAction;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.Iterator;
+ import java.util.LinkedHashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Map.Entry;
+ import java.util.NoSuchElementException;
+ import java.util.Random;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicInteger;
+ 
+ import javax.security.auth.login.LoginException;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.common.StatsSetupConst;
+ import org.apache.hadoop.hive.common.ValidTxnList;
+ import org.apache.hadoop.hive.common.ValidWriteIdList;
+ import org.apache.hadoop.hive.metastore.api.*;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+ import org.apache.hadoop.hive.metastore.hooks.URIResolverHook;
+ import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+ import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+ import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+ import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+ import org.apache.hadoop.hive.metastore.utils.ObjectPair;
+ import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
+ import org.apache.hadoop.security.UserGroupInformation;
+ import org.apache.hadoop.util.ReflectionUtils;
+ import org.apache.hadoop.util.StringUtils;
+ import org.apache.thrift.TApplicationException;
+ import org.apache.thrift.TException;
+ import org.apache.thrift.protocol.TBinaryProtocol;
+ import org.apache.thrift.protocol.TCompactProtocol;
+ import org.apache.thrift.protocol.TProtocol;
+ import org.apache.thrift.transport.TFramedTransport;
+ import org.apache.thrift.transport.TSocket;
+ import org.apache.thrift.transport.TTransport;
+ import org.apache.thrift.transport.TTransportException;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.collect.Lists;
+ 
+ /**
+  * Hive Metastore Client.
+  * The public implementation of IMetaStoreClient. Methods not inherited from 
IMetaStoreClient
+  * are not public and can change. Hence this is marked as unstable.
+  * For users who require retry mechanism when the connection between 
metastore and client is
+  * broken, RetryingMetaStoreClient class should be used.
+  */
+ @InterfaceAudience.Publi

[48/50] [abbrv] hive git commit: HIVE-19532 : fix tests for master-txnstats branch - fix one more out (Sergey Shelukhin)

2018-07-25 Thread sershe
HIVE-19532 : fix tests for master-txnstats branch - fix one more out  (Sergey 
Shelukhin)


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

Branch: refs/heads/master
Commit: 7bd688b27a6988c9a242015eeb4450784c164049
Parents: 9f9ae73
Author: sergey 
Authored: Tue Jul 24 12:39:25 2018 -0700
Committer: sergey 
Committed: Tue Jul 24 12:39:25 2018 -0700

--
 .../test/queries/clientpositive/stats_part2.q   | 12 +++--
 .../results/clientpositive/stats_part2.q.out| 28 ++--
 2 files changed, 17 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/7bd688b2/ql/src/test/queries/clientpositive/stats_part2.q
--
diff --git a/ql/src/test/queries/clientpositive/stats_part2.q 
b/ql/src/test/queries/clientpositive/stats_part2.q
index 24be218..068e928 100644
--- a/ql/src/test/queries/clientpositive/stats_part2.q
+++ b/ql/src/test/queries/clientpositive/stats_part2.q
@@ -15,6 +15,8 @@ set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 set hive.query.results.cache.enabled=false;
 
+set metastore.aggregate.stats.cache.enabled=false;
+
 -- create source.
 drop table if exists mysource;
 create table mysource (p int, key int, value string);
@@ -22,22 +24,14 @@ insert into mysource values (100,20,'value20'), 
(101,40,'string40'), (102,50,'st
 insert into mysource values (100,21,'value21'), (101,41,'value41'), 
(102,51,'value51');
 
 -- test partitioned table
-drop table if exists stats_partitioned;
+drop table if exists stats_part;
 
---create table stats_part(key int,value string) partitioned by (p int) stored 
as orc;
 create table stats_part(key int,value string) partitioned by (p int) stored as 
orc tblproperties ("transactional"="true");
---create table stats_part(key int,value string) partitioned by (p int) stored 
as orc tblproperties ("transactional"="true", 
"transactional_properties"="insert_only");
 
---explain select count(*) from stats_part;
---select count(*) from stats_part;
---explain select count(*) from stats_part where p = 100;
---select count(*) from stats_part where p = 100;
 explain select count(*) from stats_part where p > 100;
 explain select max(key) from stats_part where p > 100;
---select count(*) from stats_part where p > 100;
 desc formatted stats_part;
 
---explain insert into table stats_part partition(p=100) select distinct key, 
value from mysource where p == 100;
 insert into table stats_part partition(p=100) select distinct key, value from 
mysource where p == 100;
 insert into table stats_part partition(p=101) select distinct key, value from 
mysource where p == 101;
 insert into table stats_part partition(p=102) select distinct key, value from 
mysource where p == 102;

http://git-wip-us.apache.org/repos/asf/hive/blob/7bd688b2/ql/src/test/results/clientpositive/stats_part2.q.out
--
diff --git a/ql/src/test/results/clientpositive/stats_part2.q.out 
b/ql/src/test/results/clientpositive/stats_part2.q.out
index 9c22ce7..dfdf7b3 100644
--- a/ql/src/test/results/clientpositive/stats_part2.q.out
+++ b/ql/src/test/results/clientpositive/stats_part2.q.out
@@ -32,9 +32,9 @@ POSTHOOK: Output: default@mysource
 POSTHOOK: Lineage: mysource.key SCRIPT []
 POSTHOOK: Lineage: mysource.p SCRIPT []
 POSTHOOK: Lineage: mysource.value SCRIPT []
-PREHOOK: query: drop table if exists stats_partitioned
+PREHOOK: query: drop table if exists stats_part
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: drop table if exists stats_partitioned
+POSTHOOK: query: drop table if exists stats_part
 POSTHOOK: type: DROPTABLE
 PREHOOK: query: create table stats_part(key int,value string) partitioned by 
(p int) stored as orc tblproperties ("transactional"="true")
 PREHOOK: type: CREATETABLE
@@ -594,19 +594,19 @@ STAGE PLANS:
   TableScan
 alias: stats_part
 filterExpr: (p > 100) (type: boolean)
-Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column 
stats: PARTIAL
+Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column 
stats: COMPLETE
 Select Operator
   expressions: key (type: int)
   outputColumnNames: key
-  Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE 
Column stats: PARTIAL
+  Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE 
Column stats: COMPLETE
   Group By Operator
 aggregations: max(key)
 mode: hash
 outputColumnNames: _col0
-

[10/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
--
diff --cc 
standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
index 000,5d1a525..caa55d7
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
+++ 
b/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
@@@ -1,0 -1,1866 +1,1868 @@@
+ --
+ -- PostgreSQL database dump
+ --
+ 
+ SET statement_timeout = 0;
+ SET client_encoding = 'UTF8';
+ SET standard_conforming_strings = off;
+ SET check_function_bodies = false;
+ SET client_min_messages = warning;
+ SET escape_string_warning = off;
+ 
+ SET search_path = public, pg_catalog;
+ 
+ SET default_tablespace = '';
+ 
+ SET default_with_oids = false;
+ 
+ --
+ -- Name: BUCKETING_COLS; Type: TABLE; Schema: public; Owner: hiveuser; 
Tablespace:
+ --
+ 
+ CREATE TABLE "BUCKETING_COLS" (
+ "SD_ID" bigint NOT NULL,
+ "BUCKET_COL_NAME" character varying(256) DEFAULT NULL::character varying,
+ "INTEGER_IDX" bigint NOT NULL
+ );
+ 
+ 
+ --
+ -- Name: CDS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+ --
+ 
+ CREATE TABLE "CDS" (
+ "CD_ID" bigint NOT NULL
+ );
+ 
+ 
+ --
+ -- Name: COLUMNS_V2; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+ --
+ 
+ CREATE TABLE "COLUMNS_V2" (
+ "CD_ID" bigint NOT NULL,
+ "COMMENT" character varying(4000),
+ "COLUMN_NAME" character varying(767) NOT NULL,
+ "TYPE_NAME" text,
+ "INTEGER_IDX" integer NOT NULL
+ );
+ 
+ 
+ --
+ -- Name: DATABASE_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; 
Tablespace:
+ --
+ 
+ CREATE TABLE "DATABASE_PARAMS" (
+ "DB_ID" bigint NOT NULL,
+ "PARAM_KEY" character varying(180) NOT NULL,
+ "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+ );
+ 
+ 
+ CREATE TABLE "CTLGS" (
+ "CTLG_ID" BIGINT PRIMARY KEY,
+ "NAME" VARCHAR(256) UNIQUE,
+ "DESC" VARCHAR(4000),
+ "LOCATION_URI" VARCHAR(4000) NOT NULL
+ );
+ 
+ --
+ -- Name: DBS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+ --
+ 
+ CREATE TABLE "DBS" (
+ "DB_ID" bigint NOT NULL,
+ "DESC" character varying(4000) DEFAULT NULL::character varying,
+ "DB_LOCATION_URI" character varying(4000) NOT NULL,
+ "NAME" character varying(128) DEFAULT NULL::character varying,
+ "OWNER_NAME" character varying(128) DEFAULT NULL::character varying,
+ "OWNER_TYPE" character varying(10) DEFAULT NULL::character varying,
+ "CTLG_NAME" varchar(256)
+ );
+ 
+ 
+ --
+ -- Name: DB_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+ --
+ 
+ CREATE TABLE "DB_PRIVS" (
+ "DB_GRANT_ID" bigint NOT NULL,
+ "CREATE_TIME" bigint NOT NULL,
+ "DB_ID" bigint,
+ "GRANT_OPTION" smallint NOT NULL,
+ "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+ "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+ "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "DB_PRIV" character varying(128) DEFAULT NULL::character varying,
+ "AUTHORIZER" character varying(128) DEFAULT NULL::character varying
+ );
+ 
+ 
+ --
+ -- Name: GLOBAL_PRIVS; Type: TABLE; Schema: public; Owner: hiveuser; 
Tablespace:
+ --
+ 
+ CREATE TABLE "GLOBAL_PRIVS" (
+ "USER_GRANT_ID" bigint NOT NULL,
+ "CREATE_TIME" bigint NOT NULL,
+ "GRANT_OPTION" smallint NOT NULL,
+ "GRANTOR" character varying(128) DEFAULT NULL::character varying,
+ "GRANTOR_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "PRINCIPAL_NAME" character varying(128) DEFAULT NULL::character varying,
+ "PRINCIPAL_TYPE" character varying(128) DEFAULT NULL::character varying,
+ "USER_PRIV" character varying(128) DEFAULT NULL::character varying,
+ "AUTHORIZER" character varying(128) DEFAULT NULL::character varying
+ );
+ 
+ 
+ --
+ -- Name: IDXS; Type: TABLE; Schema: public; Owner: hiveuser; Tablespace:
+ --
+ 
+ CREATE TABLE "IDXS" (
+ "INDEX_ID" bigint NOT NULL,
+ "CREATE_TIME" bigint NOT NULL,
+ "DEFERRED_REBUILD" boolean NOT NULL,
+ "INDEX_HANDLER_CLASS" character varying(4000) DEFAULT NULL::character 
varying,
+ "INDEX_NAME" character varying(128) DEFAULT NULL::character varying,
+ "INDEX_TBL_ID" bigint,
+ "LAST_ACCESS_TIME" bigint NOT NULL,
+ "ORIG_TBL_ID" bigint,
+ "SD_ID" bigint
+ );
+ 
+ 
+ --
+ -- Name: INDEX_PARAMS; Type: TABLE; Schema: public; Owner: hiveuser; 
Tablespace:
+ --
+ 
+ CREATE TABLE "INDEX_PARAMS" (
+ "INDEX_ID" bigint NOT NULL,
+ "PARAM_KEY" character varying(256) NOT NULL,
+ "PARAM_VALUE" character varying(4000) DEFAULT NULL::character varying
+ );
+ 
+ 
+ -

[11/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
--
diff --cc 
standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
index 000,e58ee33..e985366
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
+++ 
b/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
@@@ -1,0 -1,1175 +1,1177 @@@
+ -- Table SEQUENCE_TABLE is an internal table required by DataNucleus.
+ -- NOTE: Some versions of SchemaTool do not automatically generate this table.
+ -- See http://www.datanucleus.org/servlet/jira/browse/NUCRDBMS-416
+ CREATE TABLE SEQUENCE_TABLE
+ (
+SEQUENCE_NAME VARCHAR2(255) NOT NULL,
+NEXT_VAL NUMBER NOT NULL
+ );
+ 
+ ALTER TABLE SEQUENCE_TABLE ADD CONSTRAINT PART_TABLE_PK PRIMARY KEY 
(SEQUENCE_NAME);
+ 
+ INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES 
('org.apache.hadoop.hive.metastore.model.MNotificationLog', 1);
+ 
+ -- Table NUCLEUS_TABLES is an internal table required by DataNucleus.
+ -- This table is required if datanucleus.autoStartMechanism=SchemaTable
+ -- NOTE: Some versions of SchemaTool do not automatically generate this table.
+ -- See http://www.datanucleus.org/servlet/jira/browse/NUCRDBMS-416
+ CREATE TABLE NUCLEUS_TABLES
+ (
+CLASS_NAME VARCHAR2(128) NOT NULL,
+TABLE_NAME VARCHAR2(128) NOT NULL,
+TYPE VARCHAR2(4) NOT NULL,
+OWNER VARCHAR2(2) NOT NULL,
+VERSION VARCHAR2(20) NOT NULL,
+INTERFACE_NAME VARCHAR2(255) NULL
+ );
+ 
+ ALTER TABLE NUCLEUS_TABLES ADD CONSTRAINT NUCLEUS_TABLES_PK PRIMARY KEY 
(CLASS_NAME);
+ 
+ -- Table PART_COL_PRIVS for classes 
[org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege]
+ CREATE TABLE PART_COL_PRIVS
+ (
+ PART_COLUMN_GRANT_ID NUMBER NOT NULL,
+ "COLUMN_NAME" VARCHAR2(767) NULL,
+ CREATE_TIME NUMBER (10) NOT NULL,
+ GRANT_OPTION NUMBER (5) NOT NULL,
+ GRANTOR VARCHAR2(128) NULL,
+ GRANTOR_TYPE VARCHAR2(128) NULL,
+ PART_ID NUMBER NULL,
+ PRINCIPAL_NAME VARCHAR2(128) NULL,
+ PRINCIPAL_TYPE VARCHAR2(128) NULL,
+ PART_COL_PRIV VARCHAR2(128) NULL,
+ AUTHORIZER VARCHAR2(128) NULL
+ );
+ 
+ ALTER TABLE PART_COL_PRIVS ADD CONSTRAINT PART_COL_PRIVS_PK PRIMARY KEY 
(PART_COLUMN_GRANT_ID);
+ 
+ -- Table CDS.
+ CREATE TABLE CDS
+ (
+ CD_ID NUMBER NOT NULL
+ );
+ 
+ ALTER TABLE CDS ADD CONSTRAINT CDS_PK PRIMARY KEY (CD_ID);
+ 
+ -- Table COLUMNS_V2 for join relationship
+ CREATE TABLE COLUMNS_V2
+ (
+ CD_ID NUMBER NOT NULL,
+ "COMMENT" VARCHAR2(256) NULL,
+ "COLUMN_NAME" VARCHAR2(767) NOT NULL,
+ TYPE_NAME CLOB NOT NULL,
+ INTEGER_IDX NUMBER(10) NOT NULL
+ );
+ 
+ ALTER TABLE COLUMNS_V2 ADD CONSTRAINT COLUMNS_V2_PK PRIMARY KEY 
(CD_ID,"COLUMN_NAME");
+ 
+ -- Table PARTITION_KEY_VALS for join relationship
+ CREATE TABLE PARTITION_KEY_VALS
+ (
+ PART_ID NUMBER NOT NULL,
+ PART_KEY_VAL VARCHAR2(256) NULL,
+ INTEGER_IDX NUMBER(10) NOT NULL
+ );
+ 
+ ALTER TABLE PARTITION_KEY_VALS ADD CONSTRAINT PARTITION_KEY_VALS_PK PRIMARY 
KEY (PART_ID,INTEGER_IDX);
+ 
+ CREATE TABLE CTLGS (
+ CTLG_ID NUMBER PRIMARY KEY,
+ "NAME" VARCHAR2(256),
+ "DESC" VARCHAR2(4000),
+ LOCATION_URI VARCHAR2(4000) NOT NULL,
+ UNIQUE ("NAME")
+ );
+ 
+ -- Table DBS for classes [org.apache.hadoop.hive.metastore.model.MDatabase]
+ CREATE TABLE DBS
+ (
+ DB_ID NUMBER NOT NULL,
+ "DESC" VARCHAR2(4000) NULL,
+ DB_LOCATION_URI VARCHAR2(4000) NOT NULL,
+ "NAME" VARCHAR2(128) NULL,
+ OWNER_NAME VARCHAR2(128) NULL,
+ OWNER_TYPE VARCHAR2(10) NULL,
+ CTLG_NAME VARCHAR2(256)
+ );
+ 
+ ALTER TABLE DBS ADD CONSTRAINT DBS_PK PRIMARY KEY (DB_ID);
+ 
+ -- Table PARTITION_PARAMS for join relationship
+ CREATE TABLE PARTITION_PARAMS
+ (
+ PART_ID NUMBER NOT NULL,
+ PARAM_KEY VARCHAR2(256) NOT NULL,
+ PARAM_VALUE VARCHAR2(4000) NULL
+ );
+ 
+ ALTER TABLE PARTITION_PARAMS ADD CONSTRAINT PARTITION_PARAMS_PK PRIMARY KEY 
(PART_ID,PARAM_KEY);
+ 
+ -- Table SERDES for classes 
[org.apache.hadoop.hive.metastore.model.MSerDeInfo]
+ CREATE TABLE SERDES
+ (
+ SERDE_ID NUMBER NOT NULL,
+ "NAME" VARCHAR2(128) NULL,
+ SLIB VARCHAR2(4000) NULL,
+ "DESCRIPTION" VARCHAR2(4000),
+ "SERIALIZER_CLASS" VARCHAR2(4000),
+ "DESERIALIZER_CLASS" VARCHAR2(4000),
+ "SERDE_TYPE" NUMBER
+ );
+ 
+ ALTER TABLE SERDES ADD CONSTRAINT SERDES_PK PRIMARY KEY (SERDE_ID);
+ 
+ -- Table TYPES for classes [org.apache.hadoop.hive.metastore.model.MType]
+ CREATE TABLE TYPES
+ (
+ TYPES_ID NUMBER NOT NULL,
+ TYPE_NAME VARCHAR2(128) NULL,
+ TYPE1 VARCHAR2(767) NULL,
+ TYPE2 VARCHAR2(767) NULL
+ );
+ 
+ ALTER TABLE TYPES ADD CONSTRAINT TYPES_PK PRIMARY KEY (TYPES_ID);
+ 
+ -- Table PARTITION_KEYS for join relationship
+ CREATE TABLE PAR

[32/50] [abbrv] hive git commit: HIVE-20047 : remove txnID argument for txn stats methods (Sergey Shelukhin)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
--
diff --git 
a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
 
b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 923fe2c..7fc1e43 100644
--- 
a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ 
b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -2510,7 +2510,6 @@ class TruncateTableRequest:
- dbName
- tableName
- partNames
-   - txnId
- writeId
- validWriteIdList
   """
@@ -2520,16 +2519,14 @@ class TruncateTableRequest:
 (1, TType.STRING, 'dbName', None, None, ), # 1
 (2, TType.STRING, 'tableName', None, None, ), # 2
 (3, TType.LIST, 'partNames', (TType.STRING,None), None, ), # 3
-(4, TType.I64, 'txnId', None, -1, ), # 4
-(5, TType.I64, 'writeId', None, -1, ), # 5
-(6, TType.STRING, 'validWriteIdList', None, None, ), # 6
+(4, TType.I64, 'writeId', None, -1, ), # 4
+(5, TType.STRING, 'validWriteIdList', None, None, ), # 5
   )
 
-  def __init__(self, dbName=None, tableName=None, partNames=None, 
txnId=thrift_spec[4][4], writeId=thrift_spec[5][4], validWriteIdList=None,):
+  def __init__(self, dbName=None, tableName=None, partNames=None, 
writeId=thrift_spec[4][4], validWriteIdList=None,):
 self.dbName = dbName
 self.tableName = tableName
 self.partNames = partNames
-self.txnId = txnId
 self.writeId = writeId
 self.validWriteIdList = validWriteIdList
 
@@ -2564,15 +2561,10 @@ class TruncateTableRequest:
   iprot.skip(ftype)
   elif fid == 4:
 if ftype == TType.I64:
-  self.txnId = iprot.readI64()
-else:
-  iprot.skip(ftype)
-  elif fid == 5:
-if ftype == TType.I64:
   self.writeId = iprot.readI64()
 else:
   iprot.skip(ftype)
-  elif fid == 6:
+  elif fid == 5:
 if ftype == TType.STRING:
   self.validWriteIdList = iprot.readString()
 else:
@@ -2602,16 +2594,12 @@ class TruncateTableRequest:
 oprot.writeString(iter75)
   oprot.writeListEnd()
   oprot.writeFieldEnd()
-if self.txnId is not None:
-  oprot.writeFieldBegin('txnId', TType.I64, 4)
-  oprot.writeI64(self.txnId)
-  oprot.writeFieldEnd()
 if self.writeId is not None:
-  oprot.writeFieldBegin('writeId', TType.I64, 5)
+  oprot.writeFieldBegin('writeId', TType.I64, 4)
   oprot.writeI64(self.writeId)
   oprot.writeFieldEnd()
 if self.validWriteIdList is not None:
-  oprot.writeFieldBegin('validWriteIdList', TType.STRING, 6)
+  oprot.writeFieldBegin('validWriteIdList', TType.STRING, 5)
   oprot.writeString(self.validWriteIdList)
   oprot.writeFieldEnd()
 oprot.writeFieldStop()
@@ -2630,7 +2618,6 @@ class TruncateTableRequest:
 value = (value * 31) ^ hash(self.dbName)
 value = (value * 31) ^ hash(self.tableName)
 value = (value * 31) ^ hash(self.partNames)
-value = (value * 31) ^ hash(self.txnId)
 value = (value * 31) ^ hash(self.writeId)
 value = (value * 31) ^ hash(self.validWriteIdList)
 return value
@@ -7316,7 +7303,6 @@ class SetPartitionsStatsRequest:
   Attributes:
- colStats
- needMerge
-   - txnId
- writeId
- validWriteIdList
   """
@@ -7325,15 +7311,13 @@ class SetPartitionsStatsRequest:
 None, # 0
 (1, TType.LIST, 'colStats', (TType.STRUCT,(ColumnStatistics, 
ColumnStatistics.thrift_spec)), None, ), # 1
 (2, TType.BOOL, 'needMerge', None, None, ), # 2
-(3, TType.I64, 'txnId', None, -1, ), # 3
-(4, TType.I64, 'writeId', None, -1, ), # 4
-(5, TType.STRING, 'validWriteIdList', None, None, ), # 5
+(3, TType.I64, 'writeId', None, -1, ), # 3
+(4, TType.STRING, 'validWriteIdList', None, None, ), # 4
   )
 
-  def __init__(self, colStats=None, needMerge=None, txnId=thrift_spec[3][4], 
writeId=thrift_spec[4][4], validWriteIdList=None,):
+  def __init__(self, colStats=None, needMerge=None, writeId=thrift_spec[3][4], 
validWriteIdList=None,):
 self.colStats = colStats
 self.needMerge = needMerge
-self.txnId = txnId
 self.writeId = writeId
 self.validWriteIdList = validWriteIdList
 
@@ -7364,15 +7348,10 @@ class SetPartitionsStatsRequest:
   iprot.skip(ftype)
   elif fid == 3:
 if ftype == TType.I64:
-  self.txnId = iprot.readI64()
-else:
-  iprot.skip(ftype)
-  elif fid == 4:
-if ftype == TType.I64:
   self.writeId = iprot.readI64()
 else:
   iprot.skip(ftype)
-  elif fid == 5:
+  elif fid == 4:
 if ftype == TType.STRING:
   self.validWriteIdList = iprot.readString()
 else:
@@ -7398,16 +7377,12 @@ class SetPartitionsStatsReques

[34/50] [abbrv] hive git commit: HIVE-20047 : remove txnID argument for txn stats methods (Sergey Shelukhin)

2018-07-25 Thread sershe
HIVE-20047 : remove txnID argument for txn stats methods (Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: cdb32a7fbabc9baea535b94da159965eda4e23a8
Parents: 651e795
Author: sergey 
Authored: Thu Jul 19 15:48:39 2018 -0700
Committer: sergey 
Committed: Thu Jul 19 15:48:39 2018 -0700

--
 .../listener/DummyRawStoreFailEvent.java|  37 ++-
 .../metastore/SynchronizedMetaStoreClient.java  |   4 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |  23 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java|  35 +--
 .../ql/metadata/SessionHiveMetaStoreClient.java |   9 +-
 .../hive/ql/optimizer/StatsOptimizer.java   |   4 -
 .../hadoop/hive/ql/stats/ColStatsProcessor.java |   1 -
 .../hive/ql/stats/StatsUpdaterThread.java   |   4 +-
 .../apache/hadoop/hive/ql/TestTxnCommands.java  |   2 +-
 .../hive/ql/stats/TestStatsUpdaterThread.java   |  34 +--
 .../metastore/api/AddPartitionsRequest.java | 117 +
 .../metastore/api/AlterPartitionsRequest.java   | 131 ++
 .../hive/metastore/api/AlterTableRequest.java   | 131 ++
 .../hive/metastore/api/GetTableRequest.java | 113 +
 .../metastore/api/PartitionsStatsRequest.java   | 121 +
 .../metastore/api/RenamePartitionRequest.java   | 121 +
 .../api/SetPartitionsStatsRequest.java  | 131 ++
 .../hive/metastore/api/TableStatsRequest.java   | 121 +
 .../metastore/api/TruncateTableRequest.java | 131 ++
 .../src/gen/thrift/gen-php/metastore/Types.php  | 247 ++-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  | 192 +++---
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |  42 +---
 .../src/main/thrift/hive_metastore.thrift   |  33 +--
 .../hadoop/hive/metastore/AlterHandler.java |   8 +-
 .../hadoop/hive/metastore/HiveAlterHandler.java |  52 ++--
 .../hadoop/hive/metastore/HiveMetaStore.java| 133 +-
 .../hive/metastore/HiveMetaStoreClient.java |  63 ++---
 .../hadoop/hive/metastore/IHMSHandler.java  |   2 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |  30 +--
 .../hadoop/hive/metastore/ObjectStore.java  |  92 +++
 .../apache/hadoop/hive/metastore/RawStore.java  |  20 +-
 .../hive/metastore/cache/CachedStore.java   |  56 ++---
 .../DummyRawStoreControlledCommit.java  |  38 +--
 .../DummyRawStoreForJdoConnection.java  |  22 +-
 .../HiveMetaStoreClientPreCatalog.java  |  41 +--
 .../InjectableBehaviourObjectStore.java |   5 +-
 .../hive/metastore/TestHiveAlterHandler.java|   6 +-
 .../hadoop/hive/metastore/TestObjectStore.java  |   4 +-
 .../hadoop/hive/metastore/TestOldSchema.java|   2 +-
 .../hive/metastore/cache/TestCachedStore.java   |  18 +-
 .../metastore/client/TestAlterPartitions.java   |   4 +-
 41 files changed, 506 insertions(+), 1874 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
--
diff --git 
a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
 
b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 270aa6c..1c105d1 100644
--- 
a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ 
b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -268,8 +268,8 @@ public class DummyRawStoreFailEvent implements RawStore, 
Configurable {
 
   @Override
   public Table getTable(String catName, String dbName, String tableName,
-long txnId, String writeIdList) throws MetaException {
-return objectStore.getTable(catName, dbName, tableName, txnId, 
writeIdList);
+String writeIdList) throws MetaException {
+return objectStore.getTable(catName, dbName, tableName, writeIdList);
   }
 
   @Override
@@ -286,9 +286,9 @@ public class DummyRawStoreFailEvent implements RawStore, 
Configurable {
 
   @Override
   public Partition getPartition(String catName, String dbName, String 
tableName,
-List partVals, long txnId, String 
writeIdList)
+List partVals, String writeIdList)
   throws MetaException, NoSuchObjectException {
-return objectStore.getPartition(catName, dbName, tableName, partVals, 
txnId, writeIdList);
+return objectStore.getPartition(catName, dbName, tableName, partVals, 
wri

[47/50] [abbrv] hive git commit: HIVE-19532 : fix tests for master-txnstats branch - fix one more out (Sergey Shelukhin)

2018-07-25 Thread sershe
HIVE-19532 : fix tests for master-txnstats branch - fix one more out  (Sergey 
Shelukhin)


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

Branch: refs/heads/master
Commit: 9f9ae73cc2f02bcf71a90388fa8ac0d2ca5e28af
Parents: 8047dd8
Author: sergey 
Authored: Mon Jul 23 15:22:07 2018 -0700
Committer: sergey 
Committed: Mon Jul 23 15:22:07 2018 -0700

--
 .../clientpositive/llap/acid_no_buckets.q.out   | 46 ++--
 1 file changed, 4 insertions(+), 42 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/9f9ae73c/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
--
diff --git a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out 
b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
index 9762de6..4cfb1d9 100644
--- a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
@@ -269,12 +269,13 @@ Retention:0
  A masked pattern was here 
 Table Type:MANAGED_TABLE
 Table Parameters:   
+   COLUMN_STATS_ACCURATE   {\"BASIC_STATS\":\"true\"}
bucketing_version   2   
numFiles9   
numPartitions   4   
numRows 2003
rawDataSize 0   
-   totalSize   17988   
+   totalSize   18013   
transactional   true
transactional_propertiesdefault 
  A masked pattern was here 
@@ -380,12 +381,13 @@ Retention:0
  A masked pattern was here 
 Table Type:MANAGED_TABLE
 Table Parameters:   
+   COLUMN_STATS_ACCURATE   {\"BASIC_STATS\":\"true\"}
bucketing_version   2   
numFiles9   
numPartitions   4   
numRows 2003
rawDataSize 0   
-   totalSize   17988   
+   totalSize   18013   
transactional   true
transactional_propertiesdefault 
  A masked pattern was here 
@@ -538,18 +540,10 @@ POSTHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=12
 PREHOOK: query: select count(*) from srcpart_acid
 PREHOOK: type: QUERY
 PREHOOK: Input: default@srcpart_acid
-PREHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=11
-PREHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=12
-PREHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=11
-PREHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=12
  A masked pattern was here 
 POSTHOOK: query: select count(*) from srcpart_acid
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@srcpart_acid
-POSTHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=11
-POSTHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=12
-POSTHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=11
-POSTHOOK: Input: default@srcpart_acid@ds=2008-04-09/hr=12
  A masked pattern was here 
 1990
 PREHOOK: query: merge into srcpart_acid t using (select distinct ds, hr, key, 
value from srcpart_acid) s
@@ -597,12 +591,10 @@ POSTHOOK: Lineage: merge_tmp_table.val EXPRESSION 
[(srcpart_acid)t.FieldSchema(n
 PREHOOK: query: select count(*) from srcpart_acid where ds='2008-04-08' and 
hr=='12'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@srcpart_acid
-PREHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=12
  A masked pattern was here 
 POSTHOOK: query: select count(*) from srcpart_acid where ds='2008-04-08' and 
hr=='12'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@srcpart_acid
-POSTHOOK: Input: default@srcpart_acid@ds=2008-04-08/hr=12
  A masked pattern was here 
 0
 PREHOOK: query: select ds, hr, key, value from srcpart_acid where value like 
'%updated by merge'
@@ -1057,18 +1049,10 @@ POSTHOOK: Input: 
default@srcpart_acidb@ds=2008-04-09/hr=12
 PREHOOK: query: select count(*) from srcpart_acidb
 PREHOOK: type: QUERY
 PREHOOK: Input: default@srcpart_acidb
-PREHOOK: Input: default@srcpart_acidb@ds=2008-04-08/hr=11
-PREHOOK: Input: default@srcpart_acidb@ds=2008-04-08/hr=12
-PREHOOK: Input: default@srcpart_acidb@ds=2008-04-09/hr=11
-PREHOOK: Input: default@srcpart_acidb@ds=2008-04-09/hr=12
  A masked pattern was here 
 POSTHOOK: query: select count(*) from srcpart_

[23/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 000,f45b71f..07be1ba
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@@ -1,0 -1,2817 +1,2837 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import static org.apache.commons.lang.StringUtils.join;
+ import static org.apache.commons.lang.StringUtils.normalizeSpace;
+ import static org.apache.commons.lang.StringUtils.repeat;
+ import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+ 
+ import java.sql.Blob;
+ import java.sql.Clob;
+ import java.sql.Connection;
+ import java.sql.SQLException;
+ import java.sql.Statement;
+ import java.text.ParseException;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.Iterator;
+ import java.util.LinkedList;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.TreeMap;
+ import java.util.stream.Collectors;
+ 
+ import javax.jdo.PersistenceManager;
+ import javax.jdo.Query;
+ import javax.jdo.Transaction;
+ import javax.jdo.datastore.JDOConnection;
+ 
+ import org.apache.commons.lang.BooleanUtils;
+ import org.apache.commons.lang.StringUtils;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.metastore.AggregateStatsCache.AggrColStats;
+ import org.apache.hadoop.hive.metastore.api.AggrStats;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.Order;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.PrincipalType;
+ import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+ import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+ import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+ import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+ import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+ import org.apache.hadoop.hive.metastore.api.SkewedInfo;
+ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+ import org.apache.hadoop.hive.metastore.model.MConstraint;
+ import org.apache.hadoop.hive.metastore.model.MCreationMetadata;
+ import org.apache.hadoop.hive.metastore.model.MDatabase;
+ import org.apache.hadoop.hive.metastore.model.MNotificationLog;
+ import org.apache.hadoop.hive.metastore.model.MNotificationNextId;
+ import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege;
+ import org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics;
+ import org.apache.hadoop.hive.metastore.model.MPartitionPrivilege;
+ import org.apache.hadoop.hive.metastore.model.MTableColumnStatistics;
+ import org.apache.hadoop.hive.metastore.model.MWMResourcePlan;
+ import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
+ import org.apache.hadoop.hive.metastore.parser.ExpressionTree.FilterBuilder;
+ import org.apache.hadoop.hive.metastore.parser.ExpressionTree.LeafNode;
+ import

[17/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index 000,2bae133..6fcfbce
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
@@@ -1,0 -1,514 +1,621 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hive.metastore.txn;
+ 
+ import java.sql.Connection;
+ import java.sql.Driver;
+ import java.sql.PreparedStatement;
+ import java.sql.ResultSet;
+ import java.sql.ResultSetMetaData;
+ import java.sql.SQLException;
+ import java.sql.SQLTransactionRollbackException;
+ import java.sql.Statement;
+ import java.util.Properties;
+ 
+ import com.google.common.annotations.VisibleForTesting;
++import jline.internal.Log;
+ import org.apache.hadoop.conf.Configuration;
++import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
++import org.apache.zookeeper.txn.TxnHeader;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ /**
+  * Utility methods for creating and destroying txn database/schema, plus 
methods for
+  * querying against metastore tables.
+  * Placed here in a separate class so it can be shared across unit tests.
+  */
+ public final class TxnDbUtil {
+ 
+   static final private Logger LOG = 
LoggerFactory.getLogger(TxnDbUtil.class.getName());
+   private static final String TXN_MANAGER = 
"org.apache.hadoop.hive.ql.lockmgr.DbTxnManager";
+ 
+   private static int deadlockCnt = 0;
+ 
+   private TxnDbUtil() {
+ throw new UnsupportedOperationException("Can't initialize class");
+   }
+ 
+   /**
+* Set up the configuration so it will use the DbTxnManager, concurrency 
will be set to true,
+* and the JDBC configs will be set for putting the transaction and lock 
info in the embedded
+* metastore.
+*
+* @param conf HiveConf to add these values to
+*/
+   public static void setConfValues(Configuration conf) {
+ MetastoreConf.setVar(conf, ConfVars.HIVE_TXN_MANAGER, TXN_MANAGER);
+ MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, true);
+   }
+ 
+   public static void prepDb(Configuration conf) throws Exception {
+ // This is a bogus hack because it copies the contents of the SQL file
+ // intended for creating derby databases, and thus will inexorably get
+ // out of date with it.  I'm open to any suggestions on how to make this
+ // read the file in a build friendly way.
+ 
+ Connection conn = null;
+ Statement stmt = null;
+ try {
+   conn = getConnection(conf);
+   stmt = conn.createStatement();
+   stmt.execute("CREATE TABLE TXNS (" +
+   "  TXN_ID bigint PRIMARY KEY," +
+   "  TXN_STATE char(1) NOT NULL," +
+   "  TXN_STARTED bigint NOT NULL," +
+   "  TXN_LAST_HEARTBEAT bigint NOT NULL," +
+   "  TXN_USER varchar(128) NOT NULL," +
+   "  TXN_HOST varchar(128) NOT NULL," +
+   "  TXN_TYPE integer)");
+ 
+   stmt.execute("CREATE TABLE TXN_COMPONENTS (" +
+   "  TC_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID)," +
+   "  TC_DATABASE varchar(128) NOT NULL," +
+   "  TC_TABLE varchar(128)," +
+   "  TC_PARTITION varchar(767)," +
+   "  TC_OPERATION_TYPE char(1) NOT NULL," +
+   "  TC_WRITEID bigint)");
+   stmt.execute("CREATE TABLE COMPLETED_TXN_COMPONENTS (" +
+   "  CTC_TXNID bigint NOT NULL," +
+   "  CTC_DATABASE varchar(128) NOT NULL," +
+   "  CTC_TABLE varchar(128)," +
+   "  CTC_PARTITION varchar(767)," +
+   "  CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL," +
+   "  CTC_WRITEID bigint," +
+

[38/50] [abbrv] hive git commit: HIVE-20046 : remove NUM_FILES check (Sergey Shelukhin)

2018-07-25 Thread sershe
HIVE-20046 : remove NUM_FILES check (Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: ddef89525eaef9f5eeea4029475a24b891a41df7
Parents: 31ee870
Author: sergey 
Authored: Sat Jul 21 12:32:43 2018 -0700
Committer: sergey 
Committed: Sat Jul 21 12:32:43 2018 -0700

--
 .../java/org/apache/hadoop/hive/metastore/ObjectStore.java| 7 ---
 1 file changed, 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/ddef8952/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
--
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index b319e68..9c46b68 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -12473,13 +12473,6 @@ public class ObjectStore implements RawStore, 
Configurable {
   return false;
 }
 
-// TODO## NUM_FILES could also be set to 0 by invalid update. We need to 
have a negative test. Or remove this and fix stuff.
-// If the NUM_FILES of the table/partition is 0, return 'true' from this 
method.
-// Since newly initialized empty table has 0 for the parameter.
-if (Long.parseLong(statsParams.get(StatsSetupConst.NUM_FILES)) == 0) {
-  return true;
-}
-
 if (queryValidWriteIdList != null) { // Can be null when stats are being 
reset to invalid.
   ValidWriteIdList list4TheQuery = new 
ValidReaderWriteIdList(queryValidWriteIdList);
   // Just check if the write ID is valid. If it's valid (i.e. we are 
allowed to see it),



[33/50] [abbrv] hive git commit: HIVE-20047 : remove txnID argument for txn stats methods (Sergey Shelukhin)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
--
diff --git 
a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
 
b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
index b2d85f7..7e08420 100644
--- 
a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
+++ 
b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionsStatsRequest.java
@@ -43,8 +43,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField COL_NAMES_FIELD_DESC 
= new org.apache.thrift.protocol.TField("colNames", 
org.apache.thrift.protocol.TType.LIST, (short)3);
   private static final org.apache.thrift.protocol.TField PART_NAMES_FIELD_DESC 
= new org.apache.thrift.protocol.TField("partNames", 
org.apache.thrift.protocol.TType.LIST, (short)4);
   private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = 
new org.apache.thrift.protocol.TField("catName", 
org.apache.thrift.protocol.TType.STRING, (short)5);
-  private static final org.apache.thrift.protocol.TField TXN_ID_FIELD_DESC = 
new org.apache.thrift.protocol.TField("txnId", 
org.apache.thrift.protocol.TType.I64, (short)6);
-  private static final org.apache.thrift.protocol.TField 
VALID_WRITE_ID_LIST_FIELD_DESC = new 
org.apache.thrift.protocol.TField("validWriteIdList", 
org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField 
VALID_WRITE_ID_LIST_FIELD_DESC = new 
org.apache.thrift.protocol.TField("validWriteIdList", 
org.apache.thrift.protocol.TType.STRING, (short)6);
 
   private static final Map, SchemeFactory> schemes = 
new HashMap, SchemeFactory>();
   static {
@@ -57,7 +56,6 @@ import org.slf4j.LoggerFactory;
   private List colNames; // required
   private List partNames; // required
   private String catName; // optional
-  private long txnId; // optional
   private String validWriteIdList; // optional
 
   /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
@@ -67,8 +65,7 @@ import org.slf4j.LoggerFactory;
 COL_NAMES((short)3, "colNames"),
 PART_NAMES((short)4, "partNames"),
 CAT_NAME((short)5, "catName"),
-TXN_ID((short)6, "txnId"),
-VALID_WRITE_ID_LIST((short)7, "validWriteIdList");
+VALID_WRITE_ID_LIST((short)6, "validWriteIdList");
 
 private static final Map byName = new HashMap();
 
@@ -93,9 +90,7 @@ import org.slf4j.LoggerFactory;
   return PART_NAMES;
 case 5: // CAT_NAME
   return CAT_NAME;
-case 6: // TXN_ID
-  return TXN_ID;
-case 7: // VALID_WRITE_ID_LIST
+case 6: // VALID_WRITE_ID_LIST
   return VALID_WRITE_ID_LIST;
 default:
   return null;
@@ -137,9 +132,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final int __TXNID_ISSET_ID = 0;
-  private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = 
{_Fields.CAT_NAME,_Fields.TXN_ID,_Fields.VALID_WRITE_ID_LIST};
+  private static final _Fields optionals[] = 
{_Fields.CAT_NAME,_Fields.VALID_WRITE_ID_LIST};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> 
metaDataMap;
   static {
 Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new 
EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -155,8 +148,6 @@ import org.slf4j.LoggerFactory;
 new 
org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING;
 tmpMap.put(_Fields.CAT_NAME, new 
org.apache.thrift.meta_data.FieldMetaData("catName", 
org.apache.thrift.TFieldRequirementType.OPTIONAL, 
 new 
org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-tmpMap.put(_Fields.TXN_ID, new 
org.apache.thrift.meta_data.FieldMetaData("txnId", 
org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-new 
org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
 tmpMap.put(_Fields.VALID_WRITE_ID_LIST, new 
org.apache.thrift.meta_data.FieldMetaData("validWriteIdList", 
org.apache.thrift.TFieldRequirementType.OPTIONAL, 
 new 
org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
 metaDataMap = Collections.unmodifiableMap(tmpMap);
@@ -164,8 +155,6 @@ import org.slf4j.LoggerFactory;
   }
 
   public PartitionsStatsRequest() {
-this.txnId = -1L;
-
   }
 
   public PartitionsStatsRequest(
@@ -185,

[36/50] [abbrv] hive git commit: HIVE-20029 : add parallel insert, analyze, iow tests (Sergey Shelukhin)

2018-07-25 Thread sershe
HIVE-20029 : add parallel insert, analyze, iow tests (Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: f2d5ac221da8dc81292831db06a92453a5c60ff1
Parents: bdd3cec
Author: sergey 
Authored: Fri Jul 20 15:54:01 2018 -0700
Committer: sergey 
Committed: Fri Jul 20 15:54:01 2018 -0700

--
 .../apache/hadoop/hive/ql/TestTxnCommands.java  | 189 +++
 1 file changed, 189 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/f2d5ac22/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
--
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java 
b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
index 8c33f6a..3d4cb83 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -27,6 +27,11 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.curator.shaded.com.google.common.collect.Lists;
@@ -41,6 +46,7 @@ import 
org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.LockState;
 import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
@@ -60,6 +66,7 @@ import org.apache.hadoop.hive.ql.io.BucketCodec;
 import org.apache.hadoop.hive.ql.lockmgr.TestDbTxnManager2;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.thrift.TException;
 import org.junit.Assert;
 import org.junit.Ignore;
@@ -77,6 +84,7 @@ import org.slf4j.LoggerFactory;
  * Mostly uses bucketed tables
  */
 public class TestTxnCommands extends TxnCommandsBaseForTests {
+
   static final private Logger LOG = 
LoggerFactory.getLogger(TestTxnCommands.class);
   private static final String TEST_DATA_DIR = new 
File(System.getProperty("java.io.tmpdir") +
 File.separator + TestTxnCommands.class.getCanonicalName()
@@ -108,6 +116,7 @@ public class TestTxnCommands extends 
TxnCommandsBaseForTests {
 Assert.assertEquals("1", rs.get(0));
 Assert.assertEquals("5", rs.get(1));
   }
+
   @Ignore("not needed but useful for testing")
   @Test
   public void testNonAcidInsert() throws Exception {
@@ -230,6 +239,186 @@ public class TestTxnCommands extends 
TxnCommandsBaseForTests {
 msClient.close();
   }
 
+  private static final class QueryRunnable implements Runnable {
+private final CountDownLatch cdlIn, cdlOut;
+private final String query;
+private final HiveConf hiveConf;
+
+QueryRunnable(HiveConf hiveConf, String query, CountDownLatch cdlIn, 
CountDownLatch cdlOut) {
+  this.query = query;
+  this.cdlIn = cdlIn;
+  this.cdlOut = cdlOut;
+  this.hiveConf = new HiveConf(hiveConf);
+}
+
+@Override
+public void run() {
+  SessionState ss = SessionState.start(hiveConf);
+  try {
+ss.applyAuthorizationPolicy();
+  } catch (HiveException e) {
+throw new RuntimeException(e);
+  }
+  QueryState qs = new 
QueryState.Builder().withHiveConf(hiveConf).nonIsolated().build();
+  Driver d = new Driver(qs, null);
+  try {
+LOG.info("Ready to run the query: " + query);
+syncThreadStart(cdlIn, cdlOut);
+try {
+  CommandProcessorResponse cpr = d.run(query);
+  if(cpr.getResponseCode() != 0) {
+throw new RuntimeException(query + " failed: " + cpr);
+  }
+  d.getResults(new ArrayList());
+} catch (Exception e) {
+  throw new RuntimeException(e);
+}
+  } finally {
+d.close();
+  }
+}
+  }
+
+
+  private static void syncThreadStart(final CountDownLatch cdlIn, final 
CountDownLatch cdlOut) {
+cdlIn.countDown();
+try {
+  cdlOut.await();
+} catch (InterruptedException e) {
+  throw new RuntimeException(e);
+}
+  }
+
+  @T

[04/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
--
diff --cc 
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
index 000,fb4a761..fe64a91
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
@@@ -1,0 -1,471 +1,472 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.Iterator;
+ import java.util.List;
+ import java.util.Map;
+ 
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+ import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+ import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
+ import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
+ import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
+ import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
+ import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+ import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+ import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
+ import org.apache.hadoop.hive.metastore.events.DropTableEvent;
+ import org.apache.hadoop.hive.metastore.events.ListenerEvent;
+ import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
+ import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
+ import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
+ import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
+ import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
+ import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
+ import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+ import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
+ import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
+ import org.apache.hadoop.hive.metastore.events.PreEventContext;
+ import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
+ import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+ import org.junit.Assert;
+ import org.junit.Before;
+ import org.junit.Test;
+ 
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertFalse;
+ import static org.junit.Assert.assertNotSame;
+ import static org.junit.Assert.assertTrue;
+ 
+ import com.google.common.collect.Lists;
+ 
+ import org.junit.experimental.categories.Category;
+ 
+ /**
+  * TestMetaStoreEventListener. Test case for
+  * {@link org.apache.hadoop.hive.metastore.MetaStoreEventListener} and
+  * {@link org.apache.hadoop.hive.metastore.MetaStorePreEventListener}
+  */
+ @Category(MetastoreUnitTest.class)
+ public class TestMetaStoreEventListener {
+   private Configuration conf;
+   private HiveMetaStoreClient msc;
+ 
+   private static final String dbName = "hive2038";
+   private static final String tblName = "tmptbl";
+   private static final String renamed = "tmptbl2";
+   private static fin

[22/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 000,bdcbf41..9eb8424
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@@ -1,0 -1,12207 +1,12509 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import static org.apache.commons.lang.StringUtils.join;
+ import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+ import static 
org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+ 
+ import java.io.IOException;
+ import java.lang.reflect.Field;
+ import java.net.InetAddress;
+ import java.net.URI;
+ import java.nio.ByteBuffer;
+ import java.sql.Connection;
+ import java.sql.SQLException;
+ import java.sql.SQLIntegrityConstraintViolationException;
+ import java.sql.Statement;
+ import java.time.LocalDateTime;
+ import java.time.format.DateTimeFormatter;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.Comparator;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.LinkedList;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Map.Entry;
+ import java.util.Properties;
+ import java.util.Set;
+ import java.util.TreeSet;
+ import java.util.UUID;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.locks.Lock;
+ import java.util.concurrent.locks.ReentrantLock;
+ import java.util.regex.Pattern;
+ 
+ import javax.jdo.JDOCanRetryException;
+ import javax.jdo.JDODataStoreException;
+ import javax.jdo.JDOException;
+ import javax.jdo.JDOHelper;
+ import javax.jdo.JDOObjectNotFoundException;
+ import javax.jdo.PersistenceManager;
+ import javax.jdo.PersistenceManagerFactory;
+ import javax.jdo.Query;
+ import javax.jdo.Transaction;
+ import javax.jdo.datastore.DataStoreCache;
+ import javax.jdo.datastore.JDOConnection;
+ import javax.jdo.identity.IntIdentity;
+ import javax.sql.DataSource;
+ 
+ import com.google.common.base.Strings;
+ 
+ import org.apache.commons.collections.CollectionUtils;
+ import org.apache.commons.lang.ArrayUtils;
+ import org.apache.commons.lang.StringUtils;
+ import org.apache.commons.lang.exception.ExceptionUtils;
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.conf.Configurable;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.fs.Path;
 -import org.apache.hadoop.hive.common.DatabaseName;
 -import org.apache.hadoop.hive.common.StatsSetupConst;
 -import org.apache.hadoop.hive.common.TableName;
++import org.apache.hadoop.hive.common.*;
+ import 
org.apache.hadoop.hive.metastore.MetaStoreDirectSql.SqlFilterForPushdown;
 -import org.apache.hadoop.hive.metastore.api.AggrStats;
 -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 -import org.apache.hadoop.hive.metastore.api.Catalog;
 -import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 -import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 -import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 -import org.apache.hadoop.hive.metastore.api.Database;
 -import org.apache.hadoop.hive.metastore.api.FieldSchema;
 -import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 -import org.apache.hadoop.hive.metastore.api.Function;
 -import org.apache.hadoop.hive.metastore.api.FunctionType;
 -import org.apache.hadoop.hive.metastore.api.HiveObjectPr

[42/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0721-2

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0721-2


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

Branch: refs/heads/master
Commit: 2cbe1331336222bf77743d75ddd419168a03200d
Parents: 64d75a4 cce3a05
Author: sergey 
Authored: Sat Jul 21 13:46:10 2018 -0700
Committer: sergey 
Committed: Sat Jul 21 13:46:10 2018 -0700

--
 .../org/apache/hadoop/hive/llap/LlapCacheAwareFs.java   | 12 +---
 .../org/apache/hadoop/hive/ql/io/HiveInputFormat.java   |  7 ---
 2 files changed, 13 insertions(+), 6 deletions(-)
--




[05/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
--
diff --cc 
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index 000,cb32236..75ab80b
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@@ -1,0 -1,3103 +1,3102 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import java.lang.reflect.Field;
+ import java.io.IOException;
+ import java.sql.Connection;
+ import java.sql.DriverManager;
+ import java.sql.PreparedStatement;
+ import java.sql.SQLException;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.LinkedHashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.ExecutorService;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.TimeUnit;
+ 
+ import com.google.common.collect.Sets;
+ import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+ import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+ import org.apache.hadoop.hive.metastore.utils.FileUtils;
+ import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
+ import org.datanucleus.api.jdo.JDOPersistenceManager;
+ import org.datanucleus.api.jdo.JDOPersistenceManagerFactory;
+ import org.junit.Assert;
+ import org.junit.Before;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.hive.metastore.api.AggrStats;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+ import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.Function;
+ import org.apache.hadoop.hive.metastore.api.FunctionType;
+ import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
+ import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.PrincipalType;
+ import org.apache.hadoop.hive.metastore.api.ResourceType;
+ import org.apache.hadoop.hive.metastore.api.ResourceUri;
+ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+ import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ import org.apache.hadoop.hive.metastore.api.Type;
+ import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+ import org.apache.hadoop.util.StringUtils;
+ import org.apache.thrift.TException;
+ import org.junit.Test;
+ 
+ import com.google.common.collect.Lists;
+

[25/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
index 000,29c98d1..3a65f77
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/IHMSHandler.java
@@@ -1,0 -1,104 +1,109 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import java.util.List;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.conf.Configurable;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore;
+ import org.apache.hadoop.hive.metastore.txn.TxnStore;
+ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+ 
+ /**
+  * An interface wrapper for HMSHandler.  This interface contains methods that 
need to be
+  * called by internal classes but that are not part of the thrift interface.
+  */
+ @InterfaceAudience.Private
+ public interface IHMSHandler extends ThriftHiveMetastore.Iface, Configurable {
+ 
+   void init() throws MetaException;
+ 
+   /**
+* Get the id of the thread of this handler.
+* @return thread id
+*/
+   int getThreadId();
+ 
+   /**
+* Get a reference to the underlying RawStore.
+* @return the RawStore instance.
+* @throws MetaException if the creation of a new RawStore object is 
necessary but fails.
+*/
+   RawStore getMS() throws MetaException;
+ 
+   /**
+* Get a reference to the underlying TxnStore.
+* @return the TxnStore instance.
+*/
+   TxnStore getTxnHandler();
+ 
+   /**
+* Get a reference to Hive's warehouse object (the class that does all the 
physical operations).
+* @return Warehouse instance.
+*/
+   Warehouse getWh();
+ 
+   /**
+* Equivalent to get_database, but does not write to audit logs, or fire 
pre-event listeners.
+* Meant to be used for internal hive classes that don't use the thrift 
interface.
+* @param catName catalog name
+* @param name database name
+* @return database object
+* @throws NoSuchObjectException If the database does not exist.
+* @throws MetaException If another error occurs.
+*/
+   Database get_database_core(final String catName, final String name)
+   throws NoSuchObjectException, MetaException;
+ 
+   /**
+* Equivalent of get_table, but does not log audits and fire pre-event 
listener.
+* Meant to be used for calls made by other hive classes, that are not 
using the
+* thrift interface.
+* @param catName catalog name
+* @param dbname database name
+* @param name table name
+* @return Table object
+* @throws NoSuchObjectException If the table does not exist.
+* @throws MetaException  If another error occurs.
+*/
+   Table get_table_core(final String catName, final String dbname, final 
String name)
+   throws MetaException, NoSuchObjectException;
+ 
++  Table get_table_core(final String catName, final String dbname,
++   final String name, final long txnId,
++   final String writeIdList)
++  throws MetaException, NoSuchObjectException;
++
+   /**
+* Get a list of all transactional listeners.
+* @return list of listeners.
+*/
+   List getTransactionalListeners();
+ 
+   /**
+* Get a list of all non-transactional listeners.
+* @return list of non-transactional listeners.
+*/
+   List getListeners();
+ }



[46/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0723 addendum

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0723 addendum


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

Branch: refs/heads/master
Commit: 8047dd882bfa706ae96f18e2fe0a04c818850d14
Parents: d2c60f3 5e7aa09
Author: sergey 
Authored: Mon Jul 23 11:51:14 2018 -0700
Committer: sergey 
Committed: Mon Jul 23 11:51:14 2018 -0700

--
 .../hadoop/hive/ql/plan/ExplainLockDesc.java| 116 +++
 1 file changed, 116 insertions(+)
--




[45/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0723

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0723


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

Branch: refs/heads/master
Commit: d2c60f3ae18fb18b1bc50355a7740cf352cab782
Parents: e8d7cdc 90d19ac
Author: sergey 
Authored: Mon Jul 23 11:22:33 2018 -0700
Committer: sergey 
Committed: Mon Jul 23 11:22:33 2018 -0700

--
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |2 +
 .../scripts/upgrade/derby/upgrade.order.derby   |2 +
 .../scripts/upgrade/mssql/upgrade.order.mssql   |2 +
 .../scripts/upgrade/mysql/upgrade.order.mysql   |2 +
 .../scripts/upgrade/oracle/upgrade.order.oracle |2 +
 .../upgrade/postgres/upgrade.order.postgres |2 +
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |   46 +
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   16 +-
 .../exec/spark/SparkDynamicPartitionPruner.java |   25 +-
 .../hive/ql/exec/spark/SparkPlanGenerator.java  |   24 +
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |  211 ++
 .../hadoop/hive/ql/io/HiveInputFormat.java  |   12 -
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java|  221 +--
 .../apache/hadoop/hive/ql/metadata/Table.java   |4 +
 .../calcite/translator/RexNodeConverter.java|2 +-
 .../hive/ql/parse/ExplainConfiguration.java |8 +
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |2 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g|1 +
 .../apache/hadoop/hive/ql/plan/ExplainWork.java |   17 +
 .../hive/ql/udf/generic/GenericUDFIn.java   |2 +-
 .../hive/ql/udf/generic/GenericUDFUtils.java|   57 +-
 ql/src/test/queries/clientpositive/bucket7.q|2 +
 .../test/queries/clientpositive/explain_locks.q |   22 +
 .../queries/clientpositive/orc_ppd_decimal.q|   16 +-
 .../clientpositive/parquet_ppd_decimal.q|   16 +-
 .../vectorization_parquet_ppd_decimal.q |   16 +-
 .../results/clientpositive/explain_locks.q.out  |   91 +
 .../clientpositive/llap/orc_ppd_decimal.q.out   |   48 +-
 .../clientpositive/parquet_ppd_decimal.q.out|   80 +-
 .../vectorization_parquet_ppd_decimal.q.out |   80 +-
 .../server/ThreadWithGarbageCleanup.java|6 +-
 .../hadoop/hive/metastore/HiveMetaStore.java|8 +-
 .../hive/metastore/LockComponentBuilder.java|5 +
 .../hive/metastore/LockRequestBuilder.java  |   17 +
 .../hadoop/hive/metastore/ObjectStore.java  |7 +-
 .../main/sql/derby/hive-schema-3.1.0.derby.sql  |   15 -
 .../main/sql/derby/hive-schema-3.2.0.derby.sql  |  720 +++
 .../main/sql/derby/hive-schema-4.0.0.derby.sql  |2 +-
 .../sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql  |   16 -
 .../sql/derby/upgrade-3.1.0-to-3.2.0.derby.sql  |   20 +
 .../sql/derby/upgrade-3.1.0-to-4.0.0.derby.sql  |8 -
 .../sql/derby/upgrade-3.2.0-to-4.0.0.derby.sql  |8 +
 .../src/main/sql/derby/upgrade.order.derby  |3 +-
 .../main/sql/mssql/hive-schema-3.1.0.mssql.sql  |   17 -
 .../main/sql/mssql/hive-schema-3.2.0.mssql.sql  | 1284 
 .../main/sql/mssql/hive-schema-4.0.0.mssql.sql  |2 +-
 .../sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql  |   17 -
 .../sql/mssql/upgrade-3.1.0-to-3.2.0.mssql.sql  |   23 +
 .../sql/mssql/upgrade-3.1.0-to-4.0.0.mssql.sql  |   10 -
 .../sql/mssql/upgrade-3.2.0-to-4.0.0.mssql.sql  |   10 +
 .../src/main/sql/mssql/upgrade.order.mssql  |3 +-
 .../main/sql/mysql/hive-schema-3.1.0.mysql.sql  |   16 -
 .../main/sql/mysql/hive-schema-3.2.0.mysql.sql  | 1218 
 .../main/sql/mysql/hive-schema-4.0.0.mysql.sql  |2 +-
 .../sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql  |   15 -
 .../sql/mysql/upgrade-3.1.0-to-3.2.0.mysql.sql  |   22 +
 .../sql/mysql/upgrade-3.1.0-to-4.0.0.mysql.sql  |   10 -
 .../sql/mysql/upgrade-3.2.0-to-4.0.0.mysql.sql  |   10 +
 .../src/main/sql/mysql/upgrade.order.mysql  |3 +-
 .../sql/oracle/hive-schema-3.1.0.oracle.sql |   16 -
 .../sql/oracle/hive-schema-3.2.0.oracle.sql | 1175 +++
 .../sql/oracle/hive-schema-4.0.0.oracle.sql |2 +-
 .../oracle/upgrade-3.0.0-to-3.1.0.oracle.sql|   16 -
 .../oracle/upgrade-3.1.0-to-3.2.0.oracle.sql|   22 +
 .../oracle/upgrade-3.1.0-to-4.0.0.oracle.sql|9 -
 .../oracle/upgrade-3.2.0-to-4.0.0.oracle.sql|9 +
 .../src/main/sql/oracle/upgrade.order.oracle|3 +-
 .../sql/postgres/hive-schema-3.1.0.postgres.sql |   16 -
 .../sql/postgres/hive-schema-3.2.0.postgres.sql | 1866 ++
 .../sql/postgres/hive-schema-4.0.0.postgres.sql |2 +-
 .../upgrade-3.0.0-to-3.1.0.postgres.sql |   16 -
 .../upgrade-3.1.0-to-3.2.0.postgres.sql |   22 +
 .../upgrade-3.1.0-to-4.0.0.postgres.sql |   10 -
 .../upgrade-3.2.0-to-4.0.0.postgres.sql |   10 +
 .../main/sql/pos

[01/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
Repository: hive
Updated Branches:
  refs/heads/master 71c49878c -> 758ff4490


http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
--
diff --cc 
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
index 000,816a735..bf302ed
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestTablesCreateDropAlterTruncate.java
@@@ -1,0 -1,1385 +1,1400 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore.client;
+ 
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.hive.common.StatsSetupConst;
+ import org.apache.hadoop.hive.metastore.ColumnType;
+ import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+ import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+ import org.apache.hadoop.hive.metastore.TableType;
+ import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+ import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+ import org.apache.hadoop.hive.metastore.api.Catalog;
+ import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.PrincipalType;
+ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+ import org.apache.hadoop.hive.metastore.api.SkewedInfo;
+ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+ import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+ import org.apache.thrift.TApplicationException;
+ import org.apache.thrift.TException;
+ import org.apache.thrift.protocol.TProtocolException;
+ import org.apache.thrift.transport.TTransportException;
+ import org.junit.After;
+ import org.junit.Assert;
+ import org.junit.Before;
+ import org.junit.BeforeClass;
+ import org.junit.Test;
+ import org.junit.experimental.categories.Category;
+ import org.junit.runner.RunWith;
+ import org.junit.runners.Parameterized;
+ 
+ import java.io.File;
+ import java.net.URI;
+ import java.net.URISyntaxException;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ 
+ import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+ import static 
org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+ 
+ /**
+  * Test class for IMetaStoreClient API. Testing the Table related functions 
for metadata
+  * manipulation, like creating, dropping and altering tables.
+  */
+ @RunWith(Parameterized.class)
+ @Category(MetastoreCheckinTest.class)
+ public class TestTablesCreateDropAlterTruncate extends MetaStoreClientTest {
+   private static final String DEFAULT_DATABASE = "default";
+   private static final String OTHER_DATABA

[02/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
--
diff --cc 
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
index 000,54bf3d7..0aab253
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
@@@ -1,0 -1,1117 +1,1154 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore.client;
+ 
++import java.net.ProtocolException;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.List;
+ 
+ import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+ import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+ import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+ import org.apache.hadoop.hive.metastore.api.Catalog;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.Table;
+ import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+ import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+ import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
+ import org.apache.thrift.TException;
++import org.apache.thrift.protocol.TProtocolException;
+ import org.apache.thrift.transport.TTransportException;
+ 
+ import com.google.common.collect.Lists;
+ 
+ import org.junit.After;
+ import org.junit.AfterClass;
+ import org.junit.Assert;
+ import org.junit.Before;
+ import org.junit.Test;
+ import org.junit.experimental.categories.Category;
+ import org.junit.runner.RunWith;
+ import org.junit.runners.Parameterized;
+ 
+ import static java.util.stream.Collectors.joining;
+ import static 
org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertFalse;
+ import static org.junit.Assert.assertNotEquals;
+ import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
+ 
+ /**
+  * API tests for HMS client's  alterPartitions methods.
+  */
+ @RunWith(Parameterized.class)
+ @Category(MetastoreCheckinTest.class)
+ public class TestAlterPartitions extends MetaStoreClientTest {
+   private static final int NEW_CREATE_TIME = 123456789;
+   private AbstractMetaStoreService metaStore;
+   private IMetaStoreClient client;
+ 
+   private static final String DB_NAME = "testpartdb";
+   private static final String TABLE_NAME = "testparttable";
+   private static final List PARTCOL_SCHEMA = 
Lists.newArrayList("", "mm", "dd");
+ 
+   public TestAlterPartitions(String name, AbstractMetaStoreService metaStore) 
{
+ this.metaStore = metaStore;
+   }
+ 
+   @Before
+   public void setUp() throws Exception {
+ // Get new client
+ client = metaStore.getClient();
+ 
+ // Clean up the database
+ client.dropDatabase(DB_NAME, true, true, true);
+ metaStore.cleanWarehouseDirs();
+ createDB(DB_NAME);
+   }
+ 
+   @After
+   public void tearDown() throws Exception {
+ try {
+   if (client != null) {
+ try {
+   client.close();
+ } catch (Exception e) {
+   // HIVE-19729: Shallow the exceptions based on the discussion in 
the Jira
+ }
+   }
+ } finally {
+   client = null;
+ }
+   }
+ 
+   private void cre

[30/50] [abbrv] hive git commit: HIVE-20047 : remove txnID argument for txn stats methods (Sergey Shelukhin)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
--
diff --git 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
index 0aab253..f3e0ba4 100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
@@ -863,7 +863,7 @@ public class TestAlterPartitions extends 
MetaStoreClientTest {
 createTable4PartColsParts(client);
 Partition part = client.listPartitions(DB_NAME, TABLE_NAME, 
(short)-1).get(0);
 client.alter_partitions("nosuch", DB_NAME, TABLE_NAME, 
Lists.newArrayList(part), new EnvironmentContext(),
--1, null, -1);
+null, -1);
   }
 
   @Test(expected = InvalidOperationException.class)
@@ -1080,7 +1080,7 @@ public class TestAlterPartitions extends 
MetaStoreClientTest {
 
 Partition partToRename = oldParts.get(3);
 partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
-client.renamePartition("nosuch", DB_NAME, TABLE_NAME, oldValues.get(3), 
partToRename, -1, null);
+client.renamePartition("nosuch", DB_NAME, TABLE_NAME, oldValues.get(3), 
partToRename, null);
   }
 
   @Test(expected = InvalidOperationException.class)



[43/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0722

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0722


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

Branch: refs/heads/master
Commit: b17a3471c93216976a9224c2c827b72e45c9d37d
Parents: 2cbe133 92ecdd9
Author: sergey 
Authored: Sun Jul 22 21:02:43 2018 -0700
Committer: sergey 
Committed: Sun Jul 22 21:02:43 2018 -0700

--
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   | 31 
 .../apache/hive/jdbc/HivePreparedStatement.java |  3 +-
 .../org/apache/hive/jdbc/HiveStatement.java |  3 +-
 3 files changed, 34 insertions(+), 3 deletions(-)
--




[09/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
--
diff --cc 
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 000,2454479..a5e6918
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@@ -1,0 -1,1226 +1,1268 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import org.apache.hadoop.hive.common.TableName;
 -import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 -import org.apache.hadoop.hive.metastore.api.ISchemaName;
 -import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 -import org.apache.hadoop.hive.metastore.api.Catalog;
 -import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 -
 -import java.nio.ByteBuffer;
 -import java.util.ArrayList;
 -import java.util.Collections;
 -import java.util.List;
 -import java.util.Map;
 -
 -import org.apache.hadoop.conf.Configurable;
 -import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.metastore.api.AggrStats;
+ import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
++import org.apache.hadoop.hive.metastore.api.Catalog;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+ import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
++import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
+ import org.apache.hadoop.hive.metastore.api.Function;
+ import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+ import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+ import org.apache.hadoop.hive.metastore.api.ISchema;
+ import org.apache.hadoop.hive.metastore.api.InvalidInputException;
+ import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+ import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+ import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventRequest;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventsCountResponse;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+ import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
+ import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+ import org.apache.hadoop.hive.metastore.api.PrincipalType;
+ import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+ import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+ import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
+ import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+ import org.apache.hadoop.hive.metastore.api.WMTrigger;
+ import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+ import org.apache.hadoop.hive.metastore.api.Role;
+ import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+ import org.apache.hadoop.hive.metastore.api.RuntimeStat;
+ import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+ import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+ import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+ import org.apache.hadoop.hive.metastore.

[14/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/resources/package.jdo
--
diff --cc standalone-metastore/metastore-server/src/main/resources/package.jdo
index 000,5fb548c..70150da
mode 00,100644..100644
--- a/standalone-metastore/metastore-server/src/main/resources/package.jdo
+++ b/standalone-metastore/metastore-server/src/main/resources/package.jdo
@@@ -1,0 -1,1420 +1,1426 @@@
+ 
+  
+ http://java.sun.com/dtd/jdo_2_0.dtd";>
+ 
+ 
+   
+   
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+
+ 
+ 
+
+ 
+   
+   
+ 
+   
+
+ 
+   
+ 
+ 
+ 
+   
+ 
+   
+   
+ 
+ 
+   
+   
+ 
+   
+   
+ 
+   
+ 
+ 
+ 
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+ 
+ 
+   
+ 
+   
+ 
+   
+ 
+ 
+   
+ 
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+   
+   
+ 
+ 
+   
+ 
+   
+ 
+ 
+   
+ 
+ 
+   
+ 
+   
+ 
+   
+ 
+ 
+ 
+   
+ 
+   
+   
+ 
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+   
+   
+ 
+ 
+   
+ 
+   
+   
+ 
+   
+ 
+ 
+   
+ 
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+
+ 
+ 
+
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
++  
++
++  
+ 
+ 
+ 
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+   
+   
+ 
+   
+ 
+ 
+ 
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+ 
+ 
+ 
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+
+ 
+ 
+
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+ 
+ 
+ 
+   
+ 
+   
+   
+ 
+   
+ 
+ 
+ 
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+   
+   
+ 
+   
+ 
+ 
+   
+ 
+   
+ 
+   
+ 
+ 
+
+  
+
+  
+  
+
+
+  
+
+
+  
+
+ 
+ 
+   
+ 
+   
+   
+   
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+   
+ 
+   
+   
+ 
+   
+ 
+   
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+
+ 
+ 
+
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+   
+   
+ 
+ 
+   
+   
+ 
+ 
+   
+ 
+ 
+   
+ 
+   
+ 
+ 
+ 
+   
+ 
+

[40/50] [abbrv] hive git commit: HIVE-19532 : fix tests for master-txnstats branch - more tests (Sergey Shelukhin)

2018-07-25 Thread sershe
HIVE-19532 : fix tests for master-txnstats branch - more tests (Sergey 
Shelukhin)


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

Branch: refs/heads/master
Commit: ba083ed43adb89b738387c900c822548e348ccd0
Parents: 5c8b5d8
Author: sergey 
Authored: Sat Jul 21 13:42:32 2018 -0700
Committer: sergey 
Committed: Sat Jul 21 13:42:32 2018 -0700

--
 .../metastore/client/TestAlterPartitions.java   | 26 ++--
 1 file changed, 19 insertions(+), 7 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/ba083ed4/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
--
diff --git 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
index f3e0ba4..4fc3688 100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/TestAlterPartitions.java
@@ -1050,14 +1050,18 @@ public class TestAlterPartitions extends 
MetaStoreClientTest {
 client.renamePartition(DB_NAME, TABLE_NAME, Lists.newArrayList(), 
partToRename);
   }
 
-  @Test(expected = InvalidOperationException.class)
+  @Test
   public void testRenamePartitionNullOldPartList() throws Exception {
 createTable4PartColsParts(client);
 List oldParts = client.listPartitions(DB_NAME, TABLE_NAME, 
(short)-1);
 
 Partition partToRename = oldParts.get(3);
 partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
-client.renamePartition(DB_NAME, TABLE_NAME, null, partToRename);
+try {
+  client.renamePartition(DB_NAME, TABLE_NAME, null, partToRename);
+  Assert.fail("should throw");
+} catch (InvalidOperationException | TProtocolException ex) {
+}
   }
 
   @Test
@@ -1069,7 +1073,7 @@ public class TestAlterPartitions extends 
MetaStoreClientTest {
   Partition partToRename = oldParts.get(3);
   partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
   client.renamePartition(DB_NAME, TABLE_NAME, oldValues.get(3), null);
-} catch (NullPointerException | TTransportException e) {
+} catch (NullPointerException | TProtocolException e) {
 }
   }
 
@@ -1103,24 +1107,32 @@ public class TestAlterPartitions extends 
MetaStoreClientTest {
 client.renamePartition(DB_NAME, "", oldValues.get(3), partToRename);
   }
 
-  @Test(expected = MetaException.class)
+  @Test
   public void testRenamePartitionNullDbName() throws Exception {
 List> oldValues = createTable4PartColsParts(client);
 List oldParts = client.listPartitions(DB_NAME, TABLE_NAME, 
(short)-1);
 
 Partition partToRename = oldParts.get(3);
 partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
-client.renamePartition(null, TABLE_NAME, oldValues.get(3), partToRename);
+try {
+  client.renamePartition(null, TABLE_NAME, oldValues.get(3), partToRename);
+  Assert.fail("should throw");
+} catch (MetaException | TProtocolException ex) {
+}
   }
 
-  @Test(expected = MetaException.class)
+  @Test
   public void testRenamePartitionNullTblName() throws Exception {
 List> oldValues = createTable4PartColsParts(client);
 List oldParts = client.listPartitions(DB_NAME, TABLE_NAME, 
(short)-1);
 
 Partition partToRename = oldParts.get(3);
 partToRename.setValues(Lists.newArrayList("2018", "01", "16"));
-client.renamePartition(DB_NAME, null, oldValues.get(3), partToRename);
+try {
+  client.renamePartition(DB_NAME, null, oldValues.get(3), partToRename);
+  Assert.fail("should throw");
+} catch (MetaException | TProtocolException ex) {
+}
   }
 
   @Test(expected = MetaException.class)



[21/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index 000,73a518d..95e8445
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@@ -1,0 -1,1682 +1,1719 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import org.apache.hadoop.hive.common.TableName;
 -import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 -import org.apache.hadoop.hive.metastore.api.ISchemaName;
 -import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 -import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 -import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
++import org.apache.hadoop.hive.metastore.api.*;
+ 
+ import java.lang.annotation.ElementType;
+ import java.lang.annotation.Retention;
+ import java.lang.annotation.RetentionPolicy;
+ import java.lang.annotation.Target;
+ import java.nio.ByteBuffer;
+ import java.util.List;
+ import java.util.Map;
+ 
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.conf.Configurable;
 -import org.apache.hadoop.hive.metastore.api.AggrStats;
 -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 -import org.apache.hadoop.hive.metastore.api.Catalog;
 -import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 -import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 -import org.apache.hadoop.hive.metastore.api.Database;
 -import org.apache.hadoop.hive.metastore.api.FieldSchema;
 -import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 -import org.apache.hadoop.hive.metastore.api.Function;
 -import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 -import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 -import org.apache.hadoop.hive.metastore.api.ISchema;
 -import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 -import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 -import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
 -import org.apache.hadoop.hive.metastore.api.MetaException;
 -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 -import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventRequest;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest;
 -import org.apache.hadoop.hive.metastore.api.NotificationEventsCountResponse;
 -import org.apache.hadoop.hive.metastore.api.Partition;
 -import org.apache.hadoop.hive.metastore.api.PartitionEventType;
 -import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
 -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 -import org.apache.hadoop.hive.metastore.api.PrincipalType;
 -import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 -import org.apache.hadoop.hive.metastore.api.Role;
 -import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 -import org.apache.hadoop.hive.metastore.api.RuntimeStat;
 -import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
 -import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 -import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 -import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 -import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 -import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 -import org.apache.hadoop.hive.metastore.api.SchemaVersion;
 -import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 -import org.apache.hadoop.hive.metastore.api.Table;
 -import org.apache.hadoop.hive.metastore.

[41/50] [abbrv] hive git commit: HIVE-19915 : support txn stats in CachedStore (Sergey Shelukhin)

2018-07-25 Thread sershe
HIVE-19915 : support txn stats in CachedStore (Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: 64d75a43d68edd4dfe27965e99e85bb575eda705
Parents: ba083ed
Author: sergey 
Authored: Sat Jul 21 13:43:20 2018 -0700
Committer: sergey 
Committed: Sat Jul 21 13:43:20 2018 -0700

--
 .../hadoop/hive/metastore/HiveMetaStore.java|   4 +-
 .../hadoop/hive/metastore/ObjectStore.java  |  87 
 .../apache/hadoop/hive/metastore/RawStore.java  |  13 +-
 .../hive/metastore/cache/CachedStore.java   | 202 +--
 .../hive/metastore/cache/SharedCache.java   |  48 +
 .../DummyRawStoreControlledCommit.java  |  16 +-
 .../DummyRawStoreForJdoConnection.java  |  17 +-
 7 files changed, 261 insertions(+), 126 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/64d75a43/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
--
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index eb4eb1b..fb6029f 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -5831,7 +5831,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
   boolean ret = false;
   try {
-ret = getMS().updateTableColumnStatistics(colStats, validWriteIds, 
writeId);
+ret = getMS().updateTableColumnStatistics(colStats, validWriteIds, 
writeId) != null;
   } finally {
 endFunction("write_column_statistics", ret != false, null,
 colStats.getStatsDesc().getTableName());
@@ -5875,7 +5875,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 }
 List partVals = getPartValsFromName(tbl, csd.getPartName());
 return getMS().updatePartitionColumnStatistics(
-colStats, partVals, validWriteIds, writeId);
+colStats, partVals, validWriteIds, writeId) != null;
   } finally {
 endFunction("write_partition_column_statistics", ret != false, null, 
tableName);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/64d75a43/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
--
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 9c46b68..d522a4e 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -2433,7 +2433,7 @@ public class ObjectStore implements RawStore, 
Configurable {
   @Override
   public Partition getPartition(String catName, String dbName, String 
tableName,
 List part_vals,
-String writeIdList)
+String validWriteIds)
   throws NoSuchObjectException, MetaException {
 openTransaction();
 MTable table = this.getMTable(catName, dbName, tableName);
@@ -2447,13 +2447,14 @@ public class ObjectStore implements RawStore, 
Configurable {
 part.setValues(part_vals);
 // If transactional table partition, check whether the current version 
partition
 // statistics in the metastore comply with the client query's snapshot 
isolation.
+long statsWriteId = mpart.getWriteId();
 if (TxnUtils.isTransactionalTable(table.getParameters())) {
   if (!areTxnStatsSupported) {
 // Do not make persistent the following state since it is query 
specific (not global).
 StatsSetupConst.setBasicStatsState(part.getParameters(), 
StatsSetupConst.FALSE);
 LOG.info("Removed COLUMN_STATS_ACCURATE from Partition object's 
parameters.");
-  } else if (writeIdList != null) {
-if (isCurrentStatsValidForTheQuery(part, mpart.getWriteId(), 
writeIdList, false)) {
+  } else if (validWriteIds != null) {
+if (isCurrentStatsValidForTheQuery(part, statsWriteId, validWriteIds, 
false)) {
   part.setIsStatsCompliant(true);
 } el

[28/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/StatsSetupConst.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/StatsSetupConst.java
index 000,a7ca05a..35be3c4
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/StatsSetupConst.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/common/StatsSetupConst.java
@@@ -1,0 -1,335 +1,336 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hive.common;
+ 
+ import java.io.IOException;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.TreeMap;
+ 
+ import com.google.common.collect.ImmutableList;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+ 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.
+  */
+ 
+ public class StatsSetupConst {
+ 
+   protected static final Logger LOG = 
LoggerFactory.getLogger(StatsSetupConst.class.getName());
+ 
+   public enum StatDB {
+ fs {
+   @Override
+   public String getPublisher(Configuration conf) {
+ return "org.apache.hadoop.hive.ql.stats.fs.FSStatsPublisher";
+   }
+ 
+   @Override
+   public String getAggregator(Configuration conf) {
+ return "org.apache.hadoop.hive.ql.stats.fs.FSStatsAggregator";
+   }
+ },
+ custom {
+   @Override
+   public String getPublisher(Configuration conf) {
+ return MetastoreConf.getVar(conf, ConfVars.STATS_DEFAULT_PUBLISHER); }
+   @Override
+   public String getAggregator(Configuration conf) {
+ return MetastoreConf.getVar(conf,  
ConfVars.STATS_DEFAULT_AGGREGATOR); }
+ };
+ public abstract String getPublisher(Configuration conf);
+ public abstract String getAggregator(Configuration conf);
+   }
+ 
+   // statistics stored in metastore
+   /**
+* The name of the statistic Num Files to be published or gathered.
+*/
+   public static final String NUM_FILES = "numFiles";
+ 
+   /**
+* The name of the statistic Num Partitions to be published or gathered.
+*/
+   public static final String NUM_PARTITIONS = "numPartitions";
+ 
+   /**
+* The name of the statistic Total Size to be published or gathered.
+*/
+   public static final String TOTAL_SIZE = "totalSize";
+ 
+   /**
+* The name of the statistic Row Count to be published or gathered.
+*/
+   public static final String ROW_COUNT = "numRows";
+ 
+   public static final String RUN_TIME_ROW_COUNT = "runTimeNumRows";
+ 
+   /**
+* The name of the statistic Raw Data Size to be published or gathered.
+*/
+   public static final String RAW_DATA_SIZE = "rawDataSize";
+ 
+   /**
+* The name of the statistic for Number of Erasure Coded Files - to be 
published or gathered.
+*/
+   public static final String NUM_ERASURE_CODED_FILES = "numFilesErasureCoded";
+ 
+   /**
+* Temp dir for writing stats from tasks.
+*/
+   public static final String STATS_TM

[13/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql
--
diff --cc 
standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql
index 000,5ba71c4..878530a
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql
+++ 
b/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql
@@@ -1,0 -1,720 +1,720 @@@
+ -- Timestamp: 2011-09-22 15:32:02.024
+ -- Source database is: 
/home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb
+ -- Connection URL is: 
jdbc:derby:/home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb
+ -- Specified schema is: APP
+ -- appendLogs: false
+ 
+ -- --
+ -- DDL Statements for functions
+ -- --
+ 
+ CREATE FUNCTION "APP"."NUCLEUS_ASCII" (C CHAR(1)) RETURNS INTEGER LANGUAGE 
JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 
'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.ascii' ;
+ 
+ CREATE FUNCTION "APP"."NUCLEUS_MATCHES" (TEXT VARCHAR(8000),PATTERN 
VARCHAR(8000)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL 
DATA CALLED ON NULL INPUT EXTERNAL NAME 
'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.matches' ;
+ 
+ -- --
+ -- DDL Statements for tables
+ -- --
+ CREATE TABLE "APP"."DBS" (
+   "DB_ID" BIGINT NOT NULL,
+   "DESC" VARCHAR(4000),
+   "DB_LOCATION_URI" VARCHAR(4000) NOT NULL,
+   "NAME" VARCHAR(128),
+   "OWNER_NAME" VARCHAR(128),
+   "OWNER_TYPE" VARCHAR(10),
+   "CTLG_NAME" VARCHAR(256) NOT NULL
+ );
+ 
+ CREATE TABLE "APP"."TBL_PRIVS" ("TBL_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" 
INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), 
"GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" 
VARCHAR(128), "TBL_PRIV" VARCHAR(128), "TBL_ID" BIGINT, "AUTHORIZER" 
VARCHAR(128));
+ 
+ CREATE TABLE "APP"."DATABASE_PARAMS" ("DB_ID" BIGINT NOT NULL, "PARAM_KEY" 
VARCHAR(180) NOT NULL, "PARAM_VALUE" VARCHAR(4000));
+ 
+ CREATE TABLE "APP"."TBL_COL_PRIVS" ("TBL_COLUMN_GRANT_ID" BIGINT NOT NULL, 
"COLUMN_NAME" VARCHAR(767), "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" 
SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), 
"PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "TBL_COL_PRIV" 
VARCHAR(128), "TBL_ID" BIGINT, "AUTHORIZER" VARCHAR(128));
+ 
+ CREATE TABLE "APP"."SERDE_PARAMS" ("SERDE_ID" BIGINT NOT NULL, "PARAM_KEY" 
VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB);
+ 
+ CREATE TABLE "APP"."COLUMNS_V2" ("CD_ID" BIGINT NOT NULL, "COMMENT" 
VARCHAR(4000), "COLUMN_NAME" VARCHAR(767) NOT NULL, "TYPE_NAME" CLOB, 
"INTEGER_IDX" INTEGER NOT NULL);
+ 
+ CREATE TABLE "APP"."SORT_COLS" ("SD_ID" BIGINT NOT NULL, "COLUMN_NAME" 
VARCHAR(767), "ORDER" INTEGER NOT NULL, "INTEGER_IDX" INTEGER NOT NULL);
+ 
+ CREATE TABLE "APP"."CDS" ("CD_ID" BIGINT NOT NULL);
+ 
+ CREATE TABLE "APP"."PARTITION_KEY_VALS" ("PART_ID" BIGINT NOT NULL, 
"PART_KEY_VAL" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL);
+ 
+ CREATE TABLE "APP"."DB_PRIVS" ("DB_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" 
INTEGER NOT NULL, "DB_ID" BIGINT, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" 
VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), 
"PRINCIPAL_TYPE" VARCHAR(128), "DB_PRIV" VARCHAR(128), "AUTHORIZER" 
VARCHAR(128));
+ 
+ CREATE TABLE "APP"."IDXS" ("INDEX_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER 
NOT NULL, "DEFERRED_REBUILD" CHAR(1) NOT NULL, "INDEX_HANDLER_CLASS" 
VARCHAR(4000), "INDEX_NAME" VARCHAR(128), "INDEX_TBL_ID" BIGINT, 
"LAST_ACCESS_TIME" INTEGER NOT NULL, "ORIG_TBL_ID" BIGINT, "SD_ID" BIGINT);
+ 
+ CREATE TABLE "APP"."INDEX_PARAMS" ("INDEX_ID" BIGINT NOT NULL, "PARAM_KEY" 
VARCHAR(256) NOT NULL, "PARAM_VALUE" VARCHAR(4000));
+ 
 -CREATE TABLE "APP"."PARTITIONS" ("PART_ID" BIGINT NOT NULL, "CREATE_TIME" 
INTEGER NOT NULL, "LAST_ACCESS_TIME" INTEGER NOT NULL, "PART_NAME" 
VARCHAR(767), "SD_ID" BIGINT, "TBL_ID" BIGINT);
++CREATE TABLE "APP"."PARTITIONS" ("PART_ID" BIGINT NOT NULL, "CREATE_TIME" 
INTEGER NOT NULL, "LAST_ACCESS_TIME" INTEGER NOT NULL, "PART_NAME" 
VARCHAR(767), "SD_ID" BIGINT, "TBL_ID" BIGINT, "WRITE_ID" BIGINT DEFAULT 0);
+ 
+ CREATE TABLE "APP"."SERDES" ("SERDE_ID" BIGINT NOT NULL, "NAME" VARCHAR(128), 
"SLIB" VARCHAR(4000), "DESCRIPTION" VARCHAR(4000), "SERIALIZER_CLASS" 
VARCHAR(4000), "DESERIALIZER_CLASS" VARCHAR(4000), SERDE_TYPE INTEGER);
+ 
+ CREATE TABLE "APP"."PART_PRIVS" ("PART_GRANT_ID" BIGINT NOT NULL, 
"CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" 
VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PART_ID" BIGINT, "PRINCIPAL_NAME" 
VARCHAR(128), "PRINCIPAL_TYP

[12/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
--
diff --cc 
standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
index 000,d53e7fc..e81ea2c
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
+++ 
b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
@@@ -1,0 -1,1218 +1,1220 @@@
+ -- MySQL dump 10.13  Distrib 5.5.25, for osx10.6 (i386)
+ --
+ -- Host: localhostDatabase: test
+ -- --
+ -- Server version 5.5.25
+ 
+ /*!40101 SET @OLD_CHARACTER_SET_CLIENT=@@CHARACTER_SET_CLIENT */;
+ /*!40101 SET @OLD_CHARACTER_SET_RESULTS=@@CHARACTER_SET_RESULTS */;
+ /*!40101 SET @OLD_COLLATION_CONNECTION=@@COLLATION_CONNECTION */;
+ /*!40101 SET NAMES utf8 */;
+ /*!40103 SET @OLD_TIME_ZONE=@@TIME_ZONE */;
+ /*!40103 SET TIME_ZONE='+00:00' */;
+ /*!40014 SET @OLD_UNIQUE_CHECKS=@@UNIQUE_CHECKS, UNIQUE_CHECKS=0 */;
+ /*!40014 SET @OLD_FOREIGN_KEY_CHECKS=@@FOREIGN_KEY_CHECKS, 
FOREIGN_KEY_CHECKS=0 */;
+ /*!40101 SET @OLD_SQL_MODE=@@SQL_MODE, SQL_MODE='NO_AUTO_VALUE_ON_ZERO' */;
+ /*!40111 SET @OLD_SQL_NOTES=@@SQL_NOTES, SQL_NOTES=0 */;
+ 
+ --
+ -- Table structure for table `BUCKETING_COLS`
+ --
+ 
+ /*!40101 SET @saved_cs_client = @@character_set_client */;
+ /*!40101 SET character_set_client = utf8 */;
+ CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` (
+   `SD_ID` bigint(20) NOT NULL,
+   `BUCKET_COL_NAME` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin 
DEFAULT NULL,
+   `INTEGER_IDX` int(11) NOT NULL,
+   PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
+   KEY `BUCKETING_COLS_N49` (`SD_ID`),
+   CONSTRAINT `BUCKETING_COLS_FK1` FOREIGN KEY (`SD_ID`) REFERENCES `SDS` 
(`SD_ID`)
+ ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+ /*!40101 SET character_set_client = @saved_cs_client */;
+ 
+ --
+ -- Table structure for table `CDS`
+ --
+ 
+ /*!40101 SET @saved_cs_client = @@character_set_client */;
+ /*!40101 SET character_set_client = utf8 */;
+ CREATE TABLE IF NOT EXISTS `CDS` (
+   `CD_ID` bigint(20) NOT NULL,
+   PRIMARY KEY (`CD_ID`)
+ ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+ /*!40101 SET character_set_client = @saved_cs_client */;
+ 
+ --
+ -- Table structure for table `COLUMNS_V2`
+ --
+ 
+ /*!40101 SET @saved_cs_client = @@character_set_client */;
+ /*!40101 SET character_set_client = utf8 */;
+ CREATE TABLE IF NOT EXISTS `COLUMNS_V2` (
+   `CD_ID` bigint(20) NOT NULL,
+   `COMMENT` varchar(256) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+   `COLUMN_NAME` varchar(767) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+   `TYPE_NAME` MEDIUMTEXT DEFAULT NULL,
+   `INTEGER_IDX` int(11) NOT NULL,
+   PRIMARY KEY (`CD_ID`,`COLUMN_NAME`),
+   KEY `COLUMNS_V2_N49` (`CD_ID`),
+   CONSTRAINT `COLUMNS_V2_FK1` FOREIGN KEY (`CD_ID`) REFERENCES `CDS` (`CD_ID`)
+ ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+ /*!40101 SET character_set_client = @saved_cs_client */;
+ 
+ --
+ -- Table structure for table `DATABASE_PARAMS`
+ --
+ 
+ /*!40101 SET @saved_cs_client = @@character_set_client */;
+ /*!40101 SET character_set_client = utf8 */;
+ CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` (
+   `DB_ID` bigint(20) NOT NULL,
+   `PARAM_KEY` varchar(180) CHARACTER SET latin1 COLLATE latin1_bin NOT NULL,
+   `PARAM_VALUE` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT 
NULL,
+   PRIMARY KEY (`DB_ID`,`PARAM_KEY`),
+   KEY `DATABASE_PARAMS_N49` (`DB_ID`),
+   CONSTRAINT `DATABASE_PARAMS_FK1` FOREIGN KEY (`DB_ID`) REFERENCES `DBS` 
(`DB_ID`)
+ ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+ /*!40101 SET character_set_client = @saved_cs_client */;
+ 
+ CREATE TABLE `CTLGS` (
+ `CTLG_ID` BIGINT PRIMARY KEY,
+ `NAME` VARCHAR(256),
+ `DESC` VARCHAR(4000),
+ `LOCATION_URI` VARCHAR(4000) NOT NULL,
+ UNIQUE KEY `UNIQUE_CATALOG` (`NAME`)
+ ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+ 
+ 
+ --
+ -- Table structure for table `DBS`
+ --
+ 
+ /*!40101 SET @saved_cs_client = @@character_set_client */;
+ /*!40101 SET character_set_client = utf8 */;
+ CREATE TABLE IF NOT EXISTS `DBS` (
+   `DB_ID` bigint(20) NOT NULL,
+   `DESC` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+   `DB_LOCATION_URI` varchar(4000) CHARACTER SET latin1 COLLATE latin1_bin NOT 
NULL,
+   `NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
+   `OWNER_NAME` varchar(128) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT 
NULL,
+   `OWNER_TYPE` varchar(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT 
NULL,
+   `CTLG_NAME` varchar(256) NOT NULL,
+   PRIMARY KEY (`DB_ID`),
+   UNIQUE KEY `UNIQUE_DATABASE` (`NAME`, `CTLG_NAME`),
+   CONSTRAINT `CTLG_FK1` FOREIGN KEY (`CTLG_NAME`) REFERENCES `CTLGS` (`NAME`)
+ ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+ /*!40101

[08/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
--
diff --cc 
standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 000,9b79446..8270f6a
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@@ -1,0 -1,1212 +1,1249 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import org.apache.hadoop.hive.common.TableName;
+ import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+ import org.apache.hadoop.hive.metastore.api.ISchemaName;
+ import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
+ import org.apache.hadoop.hive.metastore.api.Catalog;
+ import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.Collections;
+ import java.util.List;
+ import java.util.Map;
+ 
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.metastore.api.AggrStats;
+ import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+ import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+ import org.apache.hadoop.hive.metastore.api.Database;
+ import org.apache.hadoop.hive.metastore.api.FieldSchema;
+ import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
+ import org.apache.hadoop.hive.metastore.api.Function;
+ import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+ import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+ import org.apache.hadoop.hive.metastore.api.ISchema;
+ import org.apache.hadoop.hive.metastore.api.InvalidInputException;
+ import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+ import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+ import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
+ import org.apache.hadoop.hive.metastore.api.MetaException;
+ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+ import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventRequest;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest;
+ import org.apache.hadoop.hive.metastore.api.NotificationEventsCountResponse;
+ import org.apache.hadoop.hive.metastore.api.Partition;
+ import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+ import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
+ import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+ import org.apache.hadoop.hive.metastore.api.PrincipalType;
+ import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+ import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+ import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
+ import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+ import org.apache.hadoop.hive.metastore.api.WMTrigger;
+ import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+ import org.apache.hadoop.hive.metastore.api.Role;
+ import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+ import org.apache.hadoop.hive.metastore.api.RuntimeStat;
+ import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+ import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+ import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+ import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+ import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+ import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+ import org.apache.hadoop.hive.metastore.api.Se

[16/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 000,9dd3787..7fd0642
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@@ -1,0 -1,5051 +1,5094 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hive.metastore.txn;
+ 
+ import java.io.PrintWriter;
+ import java.nio.ByteBuffer;
+ import java.sql.Connection;
+ import java.sql.Driver;
+ import java.sql.ResultSet;
+ import java.sql.SQLException;
+ import java.sql.SQLFeatureNotSupportedException;
+ import java.sql.Savepoint;
+ import java.sql.Statement;
+ import java.time.Instant;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.BitSet;
+ import java.util.Collections;
+ import java.util.Comparator;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Properties;
+ import java.util.Set;
+ import java.util.SortedSet;
+ import java.util.TreeSet;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.concurrent.Semaphore;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicInteger;
+ import java.util.concurrent.locks.ReentrantLock;
+ import java.util.regex.Pattern;
+ 
+ import javax.sql.DataSource;
+ 
+ import org.apache.commons.lang.ArrayUtils;
+ import org.apache.commons.lang.NotImplementedException;
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.common.ValidReadTxnList;
+ import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
+ import org.apache.hadoop.hive.common.ValidTxnList;
+ import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+ import org.apache.hadoop.hive.common.ValidWriteIdList;
+ import org.apache.hadoop.hive.common.classification.RetrySemantics;
+ import org.apache.hadoop.hive.metastore.DatabaseProduct;
+ import org.apache.hadoop.hive.metastore.Warehouse;
+ import org.apache.hadoop.hive.metastore.MetaStoreListenerNotifier;
+ import org.apache.hadoop.hive.metastore.TransactionalMetaStoreEventListener;
+ import org.apache.hadoop.hive.metastore.api.*;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+ import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
+ import org.apache.hadoop.hive.metastore.datasource.DataSourceProviderFactory;
+ import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
+ import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+ import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
+ import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
+ import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
+ import org.apache.hadoop.hive.metastore.messaging.EventMessage;
+ import org.apache.hadoop.hive.metastore.metrics.Metrics;
+ import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+ import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+ import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+ import org.apache.hadoop.hive.metastore.utils.StringableMap;
+ import org.apache.hadoop.util.StringUtils;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
++
+ import com.google.common.annotations.VisibleForTesting;
+ 
+ /**
+  * A handler to answer transaction related calls that come into the metastore
+  * server.
+  *
+  * Note on log messages:  Please include txnid:X and lockid info using
+  * {@link JavaUtils#txnIdToString(long)}
+  * and {@link JavaUtils#lockIdToString(long)} in 

[18/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MPartition.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MPartition.java
index 000,4a97f89..267c9e8
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MPartition.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MPartition.java
@@@ -1,0 -1,155 +1,162 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore.model;
+ 
+ import java.util.List;
+ import java.util.Map;
+ 
+ public class MPartition {
+ 
+   private String partitionName; // partitionname ==>  (key=value/)*(key=value)
+   private MTable table; 
+   private List values;
+   private int createTime;
+   private int lastAccessTime;
+   private MStorageDescriptor sd;
+   private Map parameters;
 -  
++  private long writeId;
+   
+   public MPartition() {}
+   
+   /**
+* @param partitionName
+* @param table
+* @param values
+* @param createTime
+* @param lastAccessTime
+* @param sd
+* @param parameters
+*/
+   public MPartition(String partitionName, MTable table, List values, 
int createTime,
+   int lastAccessTime, MStorageDescriptor sd, Map 
parameters) {
+ this.partitionName = partitionName;
+ this.table = table;
+ this.values = values;
+ this.createTime = createTime;
+ this.lastAccessTime = lastAccessTime;
+ this.sd = sd;
+ this.parameters = parameters;
+   }
+ 
+   /**
+* @return the lastAccessTime
+*/
+   public int getLastAccessTime() {
+ return lastAccessTime;
+   }
+ 
+   /**
+* @param lastAccessTime the lastAccessTime to set
+*/
+   public void setLastAccessTime(int lastAccessTime) {
+ this.lastAccessTime = lastAccessTime;
+   }
+ 
+   /**
+* @return the values
+*/
+   public List getValues() {
+ return values;
+   }
+ 
+   /**
+* @param values the values to set
+*/
+   public void setValues(List values) {
+ this.values = values;
+   }
+ 
+   /**
+* @return the table
+*/
+   public MTable getTable() {
+ return table;
+   }
+ 
+   /**
+* @param table the table to set
+*/
+   public void setTable(MTable table) {
+ this.table = table;
+   }
+ 
+   /**
+* @return the sd
+*/
+   public MStorageDescriptor getSd() {
+ return sd;
+   }
+ 
+   /**
+* @param sd the sd to set
+*/
+   public void setSd(MStorageDescriptor sd) {
+ this.sd = sd;
+   }
+ 
+   /**
+* @return the parameters
+*/
+   public Map getParameters() {
+ return parameters;
+   }
+ 
+   /**
+* @param parameters the parameters to set
+*/
+   public void setParameters(Map parameters) {
+ this.parameters = parameters;
+   }
+ 
+   /**
+* @return the partitionName
+*/
+   public String getPartitionName() {
+ return partitionName;
+   }
+ 
+   /**
+* @param partitionName the partitionName to set
+*/
+   public void setPartitionName(String partitionName) {
+ this.partitionName = partitionName;
+   }
+ 
+   /**
+* @return the createTime
+*/
+   public int getCreateTime() {
+ return createTime;
+   }
+ 
+   /**
+* @param createTime the createTime to set
+*/
+   public void setCreateTime(int createTime) {
+ this.createTime = createTime;
+   }
+ 
++  public long getWriteId() {
++return writeId;
++  }
++
++  public void setWriteId(long writeId) {
++this.writeId = writeId;
++  }
+ }

http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MTable.java
index 000,38ad479..deeb971
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org

[26/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 000,92e2805..70edb96
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@@ -1,0 -1,3422 +1,3597 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import static 
org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+ import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+ import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependCatalogToDbName;
+ 
+ import java.io.IOException;
+ import java.lang.reflect.Constructor;
+ import java.lang.reflect.InvocationHandler;
+ import java.lang.reflect.InvocationTargetException;
+ import java.lang.reflect.Method;
+ import java.lang.reflect.Proxy;
+ import java.net.InetAddress;
+ import java.net.URI;
+ import java.net.UnknownHostException;
+ import java.nio.ByteBuffer;
+ import java.security.PrivilegedExceptionAction;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.Iterator;
+ import java.util.LinkedHashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Map.Entry;
+ import java.util.NoSuchElementException;
+ import java.util.Random;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicInteger;
+ 
+ import javax.security.auth.login.LoginException;
+ 
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.common.StatsSetupConst;
+ import org.apache.hadoop.hive.common.ValidTxnList;
+ import org.apache.hadoop.hive.common.ValidWriteIdList;
+ import org.apache.hadoop.hive.metastore.api.*;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+ import org.apache.hadoop.hive.metastore.hooks.URIResolverHook;
+ import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+ import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+ import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+ import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+ import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+ import org.apache.hadoop.hive.metastore.utils.ObjectPair;
+ import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
+ import org.apache.hadoop.security.UserGroupInformation;
+ import org.apache.hadoop.util.ReflectionUtils;
+ import org.apache.hadoop.util.StringUtils;
+ import org.apache.thrift.TApplicationException;
+ import org.apache.thrift.TException;
+ import org.apache.thrift.protocol.TBinaryProtocol;
+ import org.apache.thrift.protocol.TCompactProtocol;
+ import org.apache.thrift.protocol.TProtocol;
+ import org.apache.thrift.transport.TFramedTransport;
+ import org.apache.thrift.transport.TSocket;
+ import org.apache.thrift.transport.TTransport;
+ import org.apache.thrift.transport.TTransportException;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ import com.google.common.collect.Lists;
+ 
+ /**
+  * Hive Metastore Client.
+  * The public implementation of IMetaStoreClient. Methods not inherited from 
IMetaStoreClient
+  * are not public and can change. Hence this is marked as unstable.
+  * For users who require retry mechanism when the connection between 
metastore and client is
+  * broken, RetryingMetaStoreClient class should be used.
+  */
+ @InterfaceAudience.Public
+ @InterfaceStability.Evolving
+ publi

[49/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0724

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0724


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

Branch: refs/heads/master
Commit: 5e7a8b59cae36ccdd70c6540cf6ab3d2dfe1e735
Parents: 7bd688b 26f4d8e
Author: sergey 
Authored: Tue Jul 24 12:40:08 2018 -0700
Committer: sergey 
Committed: Tue Jul 24 12:40:08 2018 -0700

--
 .../predicate/TestAccumuloRangeGenerator.java   |   7 +-
 .../test/resources/testconfiguration.properties |   1 +
 .../DTIColumnArithmeticDTIColumnNoConvert.txt   |   1 -
 .../DTIScalarArithmeticDTIColumnNoConvert.txt   |   1 -
 .../org/apache/hadoop/hive/ql/ErrorMsg.java |   3 +
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   7 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java|   2 +-
 .../spark/status/impl/RemoteSparkJobStatus.java | 108 +++-
 .../ql/exec/vector/VectorizationContext.java|  26 +-
 .../vector/expressions/CastDecimalToString.java |   2 +-
 .../hive/ql/index/IndexPredicateAnalyzer.java   |   2 +
 .../apache/hadoop/hive/ql/metadata/Hive.java|  22 +-
 .../calcite/translator/RexNodeConverter.java|   4 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |   3 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java  |   1 +
 .../hadoop/hive/ql/parse/TezCompiler.java   |  42 ++
 .../hadoop/hive/ql/plan/LoadTableDesc.java  |  10 +-
 .../apache/hadoop/hive/ql/plan/TableDesc.java   |   3 +
 .../apache/hadoop/hive/ql/udf/UDFToString.java  | 181 --
 .../hive/ql/udf/generic/GenericUDFToString.java |  79 +++
 .../vector/expressions/TestVectorTypeCasts.java |   6 +-
 .../queries/clientpositive/external_insert.q|  14 +
 .../clientpositive/murmur_hash_migration.q  |  61 ++
 .../clientpositive/autoColumnStats_6.q.out  |   2 +-
 .../clientpositive/bucket_map_join_spark1.q.out |   4 +-
 .../clientpositive/bucket_map_join_spark2.q.out |   4 +-
 .../clientpositive/bucket_map_join_spark3.q.out |   4 +-
 .../results/clientpositive/bucketmapjoin5.q.out |   4 +-
 .../clientpositive/bucketmapjoin_negative.q.out |   2 +-
 .../bucketmapjoin_negative2.q.out   |   2 +-
 .../bucketsortoptimize_insert_3.q.out   |   2 +-
 .../clientpositive/char_pad_convert.q.out   |   4 +-
 .../column_pruner_multiple_children.q.out   |   2 +-
 .../test/results/clientpositive/decimal_2.q.out |   4 +-
 .../clientpositive/external_insert.q.out| 158 +
 .../test/results/clientpositive/groupby12.q.out |   2 +-
 .../test/results/clientpositive/groupby5.q.out  |   2 +-
 .../clientpositive/groupby5_noskew.q.out|   2 +-
 .../results/clientpositive/groupby7_map.q.out   |   4 +-
 .../groupby7_map_multi_single_reducer.q.out |   4 +-
 .../clientpositive/groupby7_map_skew.q.out  |   4 +-
 .../clientpositive/groupby7_noskew.q.out|   4 +-
 .../groupby7_noskew_multi_single_reducer.q.out  |   4 +-
 .../test/results/clientpositive/groupby8.q.out  |   8 +-
 .../results/clientpositive/groupby8_map.q.out   |   4 +-
 .../clientpositive/groupby8_map_skew.q.out  |   4 +-
 .../clientpositive/groupby8_noskew.q.out|   4 +-
 .../test/results/clientpositive/groupby9.q.out  |  20 +-
 .../clientpositive/groupby_cube_multi_gby.q.out |   2 +-
 .../clientpositive/groupby_position.q.out   |   8 +-
 .../clientpositive/groupby_sort_1_23.q.out  |   2 +-
 .../clientpositive/groupby_sort_skew_1_23.q.out |   4 +-
 .../infer_bucket_sort_dyn_part.q.out|   2 +-
 .../infer_bucket_sort_grouping_operators.q.out  |   6 +-
 .../infer_bucket_sort_map_operators.q.out   |   6 +-
 .../infer_bucket_sort_num_buckets.q.out |   2 +-
 .../results/clientpositive/input_part10.q.out   |   2 +-
 .../clientpositive/llap/bucketmapjoin1.q.out|   4 +-
 .../clientpositive/llap/bucketmapjoin2.q.out|   6 +-
 .../clientpositive/llap/bucketmapjoin3.q.out|   4 +-
 .../clientpositive/llap/bucketmapjoin4.q.out|   4 +-
 .../llap/default_constraint.q.out   |   4 +-
 .../llap/dynamic_partition_pruning.q.out|  16 +-
 .../llap/insert_into_default_keyword.q.out  |  20 +-
 .../llap/materialized_view_rewrite_6.q.out  |   4 +-
 .../llap/multi_insert_lateral_view.q.out|  32 +-
 .../llap/murmur_hash_migration.q.out| 618 +++
 .../clientpositive/llap/orc_merge1.q.out|   6 +-
 .../clientpositive/llap/orc_merge10.q.out   |   6 +-
 .../clientpositive/llap/orc_merge2.q.out|   2 +-
 .../clientpositive/llap/orc_merge_diff_fs.q.out |   6 +-
 .../clientpositive/llap/rcfile_merge2.q.out |   2 +-
 .../llap/schema_evol_text_nonvec_part.q.out |   4 +-
 ...hema_evol_text_nonvec_part_all_complex.q.out |  16 +-
 ...l_text_nonvec_part_all_complex_llap_io.q.out |  16 +-
 .../schema_evol_text

[31/50] [abbrv] hive git commit: HIVE-20047 : remove txnID argument for txn stats methods (Sergey Shelukhin)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/cdb32a7f/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
--
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 70edb96..91c86a7 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -420,7 +420,7 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
 
   @Override
   public void alter_table(String catName, String dbName, String tbl_name, 
Table new_tbl,
-  EnvironmentContext envContext, long txnId, String validWriteIds)
+  EnvironmentContext envContext, String validWriteIds)
   throws InvalidOperationException, MetaException, TException {
 HiveMetaHook hook = getHook(new_tbl);
 if (hook != null) {
@@ -428,7 +428,6 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
 }
 AlterTableRequest req = new AlterTableRequest(dbName, tbl_name, new_tbl);
 req.setCatName(catName);
-req.setTxnId(txnId);
 req.setValidWriteIdList(validWriteIds);
 req.setEnvironmentContext(envContext);
 client.alter_table_req(req);
@@ -438,15 +437,14 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
   @Override
   public void renamePartition(final String dbname, final String tableName, 
final List part_vals,
   final Partition newPart) throws TException {
-renamePartition(getDefaultCatalog(conf), dbname, tableName, part_vals, 
newPart, -1, null);
+renamePartition(getDefaultCatalog(conf), dbname, tableName, part_vals, 
newPart, null);
   }
 
   @Override
   public void renamePartition(String catName, String dbname, String tableName, 
List part_vals,
-  Partition newPart, long txnId, String 
validWriteIds) throws TException {
+  Partition newPart, String validWriteIds) throws 
TException {
 RenamePartitionRequest req = new RenamePartitionRequest(dbname, tableName, 
part_vals, newPart);
 req.setCatName(catName);
-req.setTxnId(txnId);
 req.setValidWriteIdList(validWriteIds);
 client.rename_partition_req(req);
   }
@@ -816,35 +814,34 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
   @Override
   public Map> getPartitionColumnStatistics(
   String dbName, String tableName, List partNames, List 
colNames,
-  long txnId, String validWriteIdList)
+  String validWriteIdList)
   throws NoSuchObjectException, MetaException, TException {
 return getPartitionColumnStatistics(getDefaultCatalog(conf), dbName, 
tableName,
-partNames, colNames, txnId, validWriteIdList);
+partNames, colNames, validWriteIdList);
   }
 
   @Override
   public Map> getPartitionColumnStatistics(
   String catName, String dbName, String tableName, List partNames,
-  List colNames, long txnId, String validWriteIdList)
+  List colNames, String validWriteIdList)
   throws NoSuchObjectException, MetaException, TException {
 PartitionsStatsRequest rqst = new PartitionsStatsRequest(dbName, 
tableName, colNames,
 partNames);
 rqst.setCatName(catName);
-rqst.setTxnId(txnId);
 rqst.setValidWriteIdList(validWriteIdList);
 return client.get_partitions_statistics_req(rqst).getPartStats();
   }
 
   @Override
   public AggrStats getAggrColStatsFor(String dbName, String tblName, 
List colNames,
-  List partNames, long txnId, String writeIdList)
+  List partNames, String writeIdList)
   throws NoSuchObjectException, MetaException, TException {
 return getAggrColStatsFor(getDefaultCatalog(conf), dbName, tblName, 
colNames,
-partNames, txnId, writeIdList);  }
+partNames, writeIdList);  }
 
   @Override
   public AggrStats getAggrColStatsFor(String catName, String dbName, String 
tblName, List colNames,
-  List partNames, long txnId, String writeIdList)
+  List partNames, String writeIdList)
   throws NoSuchObjectException, MetaException, TException {
 if (colNames.isEmpty() || partNames.isEmpty()) {
   LOG.debug("Columns is empty or partNames is empty : Short-circuiting 
stats eval on client side.");
@@ -852,7 +849,6 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
 }
 PartitionsStatsRequest req = new PartitionsStatsRequest(dbName, tblName, 
colNames, partNames);
 req.setCatName(catName);
-req.setTxnId(txnId);
 req.setValidWriteIdList(writeIdList);
 return client.get_aggr_stat

[37/50] [abbrv] hive git commit: HIVE-20219 : verify that analyze and analyze for columns manage txn stats state correctly (Sergey Shelukhin)

2018-07-25 Thread sershe
HIVE-20219 : verify that analyze and analyze for columns manage txn stats state 
correctly (Sergey Shelukhin)


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

Branch: refs/heads/master
Commit: 31ee8704e0dc5f8b9ef9530dbff4d3d1332be863
Parents: f2d5ac2
Author: sergey 
Authored: Fri Jul 20 17:41:32 2018 -0700
Committer: sergey 
Committed: Fri Jul 20 17:41:32 2018 -0700

--
 ql/src/test/queries/clientpositive/acid_stats.q |  15 -
 .../test/queries/clientpositive/acid_stats5.q   |  68 ++
 .../results/clientpositive/acid_stats.q.out | 140 ---
 .../results/clientpositive/acid_stats5.q.out| 849 +++
 4 files changed, 917 insertions(+), 155 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/31ee8704/ql/src/test/queries/clientpositive/acid_stats.q
--
diff --git a/ql/src/test/queries/clientpositive/acid_stats.q 
b/ql/src/test/queries/clientpositive/acid_stats.q
index 15eb930..8fed89a 100644
--- a/ql/src/test/queries/clientpositive/acid_stats.q
+++ b/ql/src/test/queries/clientpositive/acid_stats.q
@@ -30,18 +30,3 @@ explain select count(key) from stats_part;
 
 drop table stats_part;
 
--- test the case where we insert without updating stats... just in case
-
-create table stats2(key int,value string) tblproperties 
("transactional"="true", "transactional_properties"="insert_only");
-insert into table stats2  values (1, "foo");
-explain select count(*) from stats2;
-insert into table stats2  values (2, "bar");
-explain select count(*) from stats2;
-desc formatted stats2 key;
-
-set hive.stats.autogather=false;
-set hive.stats.column.autogather=false;
-insert into table stats2  values (1, "baz");
-explain select count(*) from stats2;
-
-drop table stats2;

http://git-wip-us.apache.org/repos/asf/hive/blob/31ee8704/ql/src/test/queries/clientpositive/acid_stats5.q
--
diff --git a/ql/src/test/queries/clientpositive/acid_stats5.q 
b/ql/src/test/queries/clientpositive/acid_stats5.q
new file mode 100644
index 000..be6a581
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/acid_stats5.q
@@ -0,0 +1,68 @@
+set hive.stats.dbclass=fs;
+set hive.stats.fetch.column.stats=true;
+set datanucleus.cache.collections=false;
+
+set hive.merge.mapfiles=false;
+set hive.merge.mapredfiles=false;
+
+set hive.stats.autogather=true;
+set hive.stats.column.autogather=true;
+set hive.compute.query.using.stats=true;
+set hive.mapred.mode=nonstrict;
+set hive.explain.user=false;
+
+set hive.fetch.task.conversion=none;
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.query.results.cache.enabled=false;
+
+-- Test various scenarios where stats become invalid; verify they are invalid, 
and that analyze works.
+
+create table stats2(key int,value string) tblproperties 
("transactional"="true", "transactional_properties"="insert_only");
+insert into table stats2  values (1, "foo");
+insert into table stats2  values (2, "bar");
+explain select count(*) from stats2;
+explain select min(key) from stats2;
+
+set hive.stats.autogather=false;
+set hive.stats.column.autogather=false;
+insert into table stats2  values (3, "baz");
+set hive.stats.autogather=true;
+set hive.stats.column.autogather=true;
+desc formatted stats2;
+desc formatted stats2 key;
+explain select count(*) from stats2;
+explain select count(distinct key) from stats2;
+
+analyze table stats2 compute statistics;
+desc formatted stats2;
+desc formatted stats2 key;
+explain select count(*) from stats2;
+explain select min(key) from stats2;
+
+analyze table stats2 compute statistics for columns;
+desc formatted stats2;
+desc formatted stats2 key;
+explain select count(*) from stats2;
+explain select min(key) from stats2;
+
+
+truncate table stats2;
+desc formatted stats2;
+desc formatted stats2 key;
+explain select count(*) from stats2;
+explain select count(distinct key) from stats2;
+
+analyze table stats2 compute statistics;
+desc formatted stats2;
+desc formatted stats2 key;
+explain select count(*) from stats2;
+explain select min(key) from stats2;
+
+analyze table stats2 compute statistics for columns;
+desc formatted stats2;
+desc formatted stats2 key;
+explain select count(*) from stats2;
+explain select min(key) from stats2;
+
+drop table stats2;

http://git-wip-us.apache.org/repos/asf/hive/blob/31ee8704/ql/src/test/results/clientpositive/acid_stats.q.out
--
diff --git a/ql/src/test/results/clientpositive/acid_stats.q.out 
b/ql/src/test/re

[35/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0720

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0720


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

Branch: refs/heads/master
Commit: bdd3cec1f68178aa2128ee6db7fbc0a0c0bb021f
Parents: cdb32a7 06a4f98
Author: sergey 
Authored: Fri Jul 20 12:24:26 2018 -0700
Committer: sergey 
Committed: Fri Jul 20 12:24:26 2018 -0700

--
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  10 ++
 ql/src/test/queries/clientpositive/interval_3.q |  23 
 .../results/clientpositive/interval_3.q.out | 111 +++
 3 files changed, 144 insertions(+)
--


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



[39/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0721

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0721


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

Branch: refs/heads/master
Commit: 5c8b5d8b62beaffedd0681683638bf8bb7d98402
Parents: ddef895 788daf6
Author: sergey 
Authored: Sat Jul 21 12:51:36 2018 -0700
Committer: sergey 
Committed: Sat Jul 21 12:51:36 2018 -0700

--
 .../test/resources/testconfiguration.properties |   1 +
 .../hive/ql/optimizer/GenMRTableScan1.java  |   5 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java  |  12 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   1 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g|  11 +-
 .../hive/ql/parse/ProcessAnalyzeTable.java  |   5 +-
 .../hadoop/hive/ql/parse/TaskCompiler.java  |   6 +-
 .../parse/spark/SparkProcessAnalyzeTable.java   |   5 +-
 .../hive/ql/stats/BasicStatsNoJobTask.java  |   9 +
 .../queries/clientpositive/acid_no_buckets.q|   8 +
 .../clientpositive/truncate_external_force.q|  16 ++
 .../clientpositive/acid_table_stats.q.out   |  16 +-
 .../materialized_view_create_rewrite.q.out  |  58 ++---
 .../clientpositive/llap/acid_no_buckets.q.out   | 210 +--
 .../materialized_view_create_rewrite_4.q.out|   6 +-
 .../materialized_view_create_rewrite_5.q.out|   2 +-
 .../llap/truncate_external_force.q.out  |  85 
 17 files changed, 393 insertions(+), 63 deletions(-)
--


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

http://git-wip-us.apache.org/repos/asf/hive/blob/5c8b5d8b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStatsNoJobTask.java
--

http://git-wip-us.apache.org/repos/asf/hive/blob/5c8b5d8b/ql/src/test/results/clientpositive/acid_table_stats.q.out
--
diff --cc ql/src/test/results/clientpositive/acid_table_stats.q.out
index f3ada34,2fe4f97..32badd0
--- a/ql/src/test/results/clientpositive/acid_table_stats.q.out
+++ b/ql/src/test/results/clientpositive/acid_table_stats.q.out
@@@ -174,10 -211,9 +176,10 @@@ Database:defaul
  Table:acid 
   A masked pattern was here 
  Partition Parameters:  
 +  COLUMN_STATS_ACCURATE   
{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"key\":\"true\",\"value\":\"true\"}}
numFiles2   
numRows 1000
-   rawDataSize 208000  
+   rawDataSize 0   
totalSize   4063
   A masked pattern was here 
 
@@@ -225,10 -261,10 +227,10 @@@ Database:   defaul
  Table:acid 
   A masked pattern was here 
  Partition Parameters:  
 -  COLUMN_STATS_ACCURATE   
{\"COLUMN_STATS\":{\"key\":\"true\",\"value\":\"true\"}}
 +  COLUMN_STATS_ACCURATE   
{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"key\":\"true\",\"value\":\"true\"}}
numFiles2   
numRows 1000
-   rawDataSize 208000  
+   rawDataSize 0   
totalSize   4063
   A masked pattern was here 
 
@@@ -317,11 -389,10 +319,11 @@@ Database:   defaul
  Table:acid 
   A masked pattern was here 
  Partition Parameters:  
 +  COLUMN_STATS_ACCURATE   
{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"key\":\"true\",\"value\":\"true\"}}
numFiles4   
 -  numRows 3000
 +  numRows 2000
-   rawDataSize 208000  
+   rawDataSize 0   
 -  totalSize   8118
 +  totalSize   8126
   A masked pattern was here 
 
  # Storage Information  
@@@ -364,11 -437,10 +368,11 @@@ Database:   defaul
  Table:acid 
   A masked pattern was here 
  Partition Parameters:  
 +  COLUMN_STATS_ACCURATE   
{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"key\":\"true\",\"value\":\"true\"}}
numFiles4 

[50/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0725

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0725


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

Branch: refs/heads/master
Commit: 758ff449099065a84c46d63f9418201c8a6731b1
Parents: 5e7a8b5 71c4987
Author: sergey 
Authored: Wed Jul 25 11:18:51 2018 -0700
Committer: sergey 
Committed: Wed Jul 25 11:18:51 2018 -0700

--
 .../VectorizedComparisonBench.java  |   19 +-
 .../tezplugins/LlapTaskSchedulerService.java|   12 +-
 .../ExpressionTemplates/ColumnCompareColumn.txt |4 +-
 .../ExpressionTemplates/ColumnCompareScalar.txt |4 +-
 .../Decimal64ColumnCompareDecimal64Column.txt   |   54 +
 .../Decimal64ColumnCompareDecimal64Scalar.txt   |   66 +
 .../Decimal64ScalarCompareDecimal64Column.txt   |   66 +
 .../DecimalColumnCompareDecimalColumn.txt   |  153 +
 .../DecimalColumnCompareDecimalScalar.txt   |  177 +
 .../DecimalScalarCompareDecimalColumn.txt   |  180 +
 .../ExpressionTemplates/IfExprColumnScalar.txt  |6 +-
 .../IfExprObjectColumnColumn.txt|   41 +
 .../IfExprObjectColumnScalar.txt|   22 +
 .../IfExprObjectScalarColumn.txt|   22 +
 .../ExpressionTemplates/IfExprScalarColumn.txt  |6 +-
 .../ExpressionTemplates/IfExprScalarScalar.txt  |6 +-
 .../ExpressionTemplates/ScalarCompareColumn.txt |4 +-
 .../exec/vector/VectorExpressionDescriptor.java |2 +
 .../ql/exec/vector/VectorizationContext.java|  222 +-
 .../exec/vector/VectorizationContext.java.orig  | 3771 ++
 .../vector/expressions/CastCharToBinary.java|   55 +
 .../expressions/CastStringToTimestamp.java  |  177 +
 .../IfExprCharScalarStringScalar.java   |2 +-
 .../IfExprDecimal64ColumnDecimal64Column.java   |   55 +
 .../IfExprDecimal64ColumnDecimal64Scalar.java   |   70 +
 .../IfExprDecimal64ScalarDecimal64Column.java   |   71 +
 .../IfExprDecimal64ScalarDecimal64Scalar.java   |   75 +
 .../expressions/IfExprLongColumnLongColumn.java |6 +-
 ...fExprStringGroupColumnStringGroupColumn.java |4 +-
 .../IfExprStringGroupColumnStringScalar.java|4 +-
 .../IfExprStringScalarStringGroupColumn.java|4 +-
 .../IfExprStringScalarStringScalar.java |4 +-
 .../IfExprVarCharScalarStringScalar.java|2 +-
 .../expressions/LongColEqualLongColumn.java |  146 -
 .../expressions/LongColEqualLongScalar.java |  157 -
 .../LongColGreaterEqualLongColumn.java  |  146 -
 .../LongColGreaterEqualLongScalar.java  |  158 -
 .../expressions/LongColGreaterLongColumn.java   |  146 -
 .../expressions/LongColGreaterLongScalar.java   |  157 -
 .../expressions/LongColLessEqualLongColumn.java |  146 -
 .../expressions/LongColLessEqualLongScalar.java |  158 -
 .../expressions/LongColLessLongColumn.java  |  146 -
 .../expressions/LongColLessLongScalar.java  |  158 -
 .../expressions/LongColNotEqualLongColumn.java  |  146 -
 .../expressions/LongColNotEqualLongScalar.java  |  158 -
 .../expressions/LongScalarEqualLongColumn.java  |  157 -
 .../LongScalarGreaterEqualLongColumn.java   |  160 -
 .../LongScalarGreaterLongColumn.java|  161 -
 .../LongScalarLessEqualLongColumn.java  |  160 -
 .../expressions/LongScalarLessLongColumn.java   |  161 -
 .../LongScalarNotEqualLongColumn.java   |  161 -
 .../hive/ql/udf/generic/GenericUDFIf.java   |   16 +
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |8 +-
 .../generic/GenericUDFOPEqualOrGreaterThan.java |8 +-
 .../generic/GenericUDFOPEqualOrLessThan.java|8 +-
 .../ql/udf/generic/GenericUDFOPGreaterThan.java |8 +-
 .../ql/udf/generic/GenericUDFOPLessThan.java|8 +-
 .../ql/udf/generic/GenericUDFOPNotEqual.java|8 +-
 .../ql/udf/generic/GenericUDFTimestamp.java |3 +-
 .../exec/vector/TestVectorizationContext.java   |4 +-
 .../expressions/TestVectorArithmetic.java   |   11 +
 .../expressions/TestVectorCastStatement.java|   42 +-
 .../expressions/TestVectorDateAddSub.java   |   10 +
 .../vector/expressions/TestVectorDateDiff.java  |   11 +
 .../expressions/TestVectorFilterCompare.java|  795 
 .../expressions/TestVectorIfStatement.java  |   74 +-
 .../vector/expressions/TestVectorNegative.java  |9 +
 .../expressions/TestVectorStringConcat.java |   11 +
 .../expressions/TestVectorStringUnary.java  |   12 +-
 .../vector/expressions/TestVectorSubStr.java|9 +
 .../expressions/TestVectorTimestampExtract.java |9 +
 .../clientnegative/alter_table_wrong_db.q   |4 +
 .../clientnegative/alter_table_wrong_table.q|3 +
 .../clientnegative/alter_table_wrong_db.q.out   |   25 +
 .../alter_table_wrong_table.q.out   | 

[15/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index 000,33f24fb..080cc52
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@@ -1,0 -1,504 +1,509 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.hadoop.hive.metastore.txn;
+ 
+ import com.google.common.annotations.VisibleForTesting;
++
+ import org.apache.hadoop.classification.InterfaceAudience;
+ import org.apache.hadoop.classification.InterfaceStability;
+ import org.apache.hadoop.conf.Configurable;
+ import org.apache.hadoop.hive.common.ValidTxnList;
+ import org.apache.hadoop.hive.common.ValidWriteIdList;
+ import org.apache.hadoop.hive.common.classification.RetrySemantics;
+ import org.apache.hadoop.hive.metastore.api.*;
+ import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
+ 
+ import java.sql.SQLException;
+ import java.util.Iterator;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ 
+ /**
+  * A handler to answer transaction related calls that come into the metastore
+  * server.
+  */
+ @InterfaceAudience.Private
+ @InterfaceStability.Evolving
+ public interface TxnStore extends Configurable {
+ 
+   enum MUTEX_KEY {
+ Initiator, Cleaner, HouseKeeper, CompactionHistory, CheckLock,
+ WriteSetCleaner, CompactionScheduler, WriteIdAllocator, 
MaterializationRebuild
+   }
+   // Compactor states (Should really be enum)
+   String INITIATED_RESPONSE = "initiated";
+   String WORKING_RESPONSE = "working";
+   String CLEANING_RESPONSE = "ready for cleaning";
+   String FAILED_RESPONSE = "failed";
+   String SUCCEEDED_RESPONSE = "succeeded";
+   String ATTEMPTED_RESPONSE = "attempted";
+ 
+   int TIMED_OUT_TXN_ABORT_BATCH_SIZE = 5;
+ 
+   /**
+* Get information about open transactions.  This gives extensive 
information about the
+* transactions rather than just the list of transactions.  This should be 
used when the need
+* is to see information about the transactions (e.g. show transactions).
+* @return information about open transactions
+* @throws MetaException
+*/
+   @RetrySemantics.ReadOnly
+   GetOpenTxnsInfoResponse getOpenTxnsInfo() throws MetaException;
+ 
+   /**
+* Get list of valid transactions.  This gives just the list of 
transactions that are open.
+* @return list of open transactions, as well as a high water mark.
+* @throws MetaException
+*/
+   @RetrySemantics.ReadOnly
+   GetOpenTxnsResponse getOpenTxns() throws MetaException;
+ 
+   /**
+* Get the count for open transactions.
+* @throws MetaException
+*/
+   @RetrySemantics.ReadOnly
+   void countOpenTxns() throws MetaException;
+ 
+   /**
+* Open a set of transactions
+* @param rqst request to open transactions
+* @return information on opened transactions
+* @throws MetaException
+*/
+   @RetrySemantics.Idempotent
+   OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException;
+ 
+   @RetrySemantics.Idempotent
+   long getTargetTxnId(String replPolicy, long sourceTxnId) throws 
MetaException;
+ 
+   /**
+* Abort (rollback) a transaction.
+* @param rqst info on transaction to abort
+* @throws NoSuchTxnException
+* @throws MetaException
+*/
+   @RetrySemantics.Idempotent
+   void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, 
MetaException, TxnAbortedException;
+ 
+   /**
+* Abort (rollback) a list of transactions in one request.
+* @param rqst info on transactions to abort
+* @throws NoSuchTxnException
+* @throws MetaException
+*/
+   @RetrySemantics.Idempotent
+   void abortTxns(AbortTxnsRequest rqst) throws NoSuchTxnException, 
MetaException;
+ 
+   /**
+* Commit a transaction
+* @param rqst info on tra

[27/50] [abbrv] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0719

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/651e7950/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
--
diff --cc 
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 000,47f819b..285f7fb
mode 00,100644..100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@@ -1,0 -1,9353 +1,9602 @@@
+ /* * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  * http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.hadoop.hive.metastore;
+ 
+ import static org.apache.commons.lang.StringUtils.join;
+ import static 
org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_COMMENT;
+ import static 
org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+ import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+ import static 
org.apache.hadoop.hive.metastore.Warehouse.getCatalogQualifiedTableName;
+ import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+ import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.parseDbName;
+ import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.CAT_NAME;
+ import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.DB_NAME;
+ import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependCatalogToDbName;
+ import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependNotNullCatToDbName;
+ 
+ import java.io.IOException;
+ import java.net.InetAddress;
+ import java.net.UnknownHostException;
+ import java.nio.ByteBuffer;
+ import java.security.PrivilegedExceptionAction;
+ import java.util.AbstractMap;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.LinkedHashMap;
+ import java.util.LinkedList;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Map.Entry;
+ import java.util.Objects;
+ import java.util.Properties;
+ import java.util.Set;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.concurrent.ExecutionException;
+ import java.util.concurrent.ExecutorService;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicInteger;
+ import java.util.concurrent.locks.Condition;
+ import java.util.concurrent.locks.Lock;
+ import java.util.concurrent.locks.ReentrantLock;
+ import java.util.regex.Pattern;
+ 
+ import javax.jdo.JDOException;
+ 
+ import com.codahale.metrics.Counter;
+ import com.google.common.collect.ImmutableList;
+ import com.google.common.collect.ImmutableListMultimap;
+ import com.google.common.collect.Lists;
+ import com.google.common.collect.Multimaps;
+ 
+ import org.apache.commons.cli.OptionBuilder;
+ import org.apache.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.common.StatsSetupConst;
+ import org.apache.hadoop.hive.common.TableName;
+ import org.apache.hadoop.hive.metastore.api.*;
+ import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
+ import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.StatsUpdateMode;
+ import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
+ import org.apache.hadoop.hive.metastore.events.AddNotNullConstraintEvent;
+ import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+ import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
+ import org.apache.hadoop.hive.metastore.events.AddUniqueConstraintEven

[10/11] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread sershe
HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare 
(Matt McCline, reviewed by Teddy Choi)


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

Branch: refs/heads/master-txnstats
Commit: 71c49878c3669f19f900ec1ffe7652c91ce15d38
Parents: ca0d706
Author: Matt McCline 
Authored: Wed Jul 25 09:58:51 2018 -0500
Committer: Matt McCline 
Committed: Wed Jul 25 09:59:35 2018 -0500

--
 .../VectorizedComparisonBench.java  |   19 +-
 .../ExpressionTemplates/ColumnCompareColumn.txt |4 +-
 .../ExpressionTemplates/ColumnCompareScalar.txt |4 +-
 .../Decimal64ColumnCompareDecimal64Column.txt   |   54 +
 .../Decimal64ColumnCompareDecimal64Scalar.txt   |   66 +
 .../Decimal64ScalarCompareDecimal64Column.txt   |   66 +
 .../DecimalColumnCompareDecimalColumn.txt   |  153 +
 .../DecimalColumnCompareDecimalScalar.txt   |  177 +
 .../DecimalScalarCompareDecimalColumn.txt   |  180 +
 .../ExpressionTemplates/IfExprColumnScalar.txt  |6 +-
 .../IfExprObjectColumnColumn.txt|   41 +
 .../IfExprObjectColumnScalar.txt|   22 +
 .../IfExprObjectScalarColumn.txt|   22 +
 .../ExpressionTemplates/IfExprScalarColumn.txt  |6 +-
 .../ExpressionTemplates/IfExprScalarScalar.txt  |6 +-
 .../ExpressionTemplates/ScalarCompareColumn.txt |4 +-
 .../exec/vector/VectorExpressionDescriptor.java |2 +
 .../ql/exec/vector/VectorizationContext.java|  222 +-
 .../exec/vector/VectorizationContext.java.orig  | 3771 ++
 .../vector/expressions/CastCharToBinary.java|   55 +
 .../expressions/CastStringToTimestamp.java  |  177 +
 .../IfExprCharScalarStringScalar.java   |2 +-
 .../IfExprDecimal64ColumnDecimal64Column.java   |   55 +
 .../IfExprDecimal64ColumnDecimal64Scalar.java   |   70 +
 .../IfExprDecimal64ScalarDecimal64Column.java   |   71 +
 .../IfExprDecimal64ScalarDecimal64Scalar.java   |   75 +
 .../expressions/IfExprLongColumnLongColumn.java |6 +-
 ...fExprStringGroupColumnStringGroupColumn.java |4 +-
 .../IfExprStringGroupColumnStringScalar.java|4 +-
 .../IfExprStringScalarStringGroupColumn.java|4 +-
 .../IfExprStringScalarStringScalar.java |4 +-
 .../IfExprVarCharScalarStringScalar.java|2 +-
 .../expressions/LongColEqualLongColumn.java |  146 -
 .../expressions/LongColEqualLongScalar.java |  157 -
 .../LongColGreaterEqualLongColumn.java  |  146 -
 .../LongColGreaterEqualLongScalar.java  |  158 -
 .../expressions/LongColGreaterLongColumn.java   |  146 -
 .../expressions/LongColGreaterLongScalar.java   |  157 -
 .../expressions/LongColLessEqualLongColumn.java |  146 -
 .../expressions/LongColLessEqualLongScalar.java |  158 -
 .../expressions/LongColLessLongColumn.java  |  146 -
 .../expressions/LongColLessLongScalar.java  |  158 -
 .../expressions/LongColNotEqualLongColumn.java  |  146 -
 .../expressions/LongColNotEqualLongScalar.java  |  158 -
 .../expressions/LongScalarEqualLongColumn.java  |  157 -
 .../LongScalarGreaterEqualLongColumn.java   |  160 -
 .../LongScalarGreaterLongColumn.java|  161 -
 .../LongScalarLessEqualLongColumn.java  |  160 -
 .../expressions/LongScalarLessLongColumn.java   |  161 -
 .../LongScalarNotEqualLongColumn.java   |  161 -
 .../hive/ql/udf/generic/GenericUDFIf.java   |   16 +
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |8 +-
 .../generic/GenericUDFOPEqualOrGreaterThan.java |8 +-
 .../generic/GenericUDFOPEqualOrLessThan.java|8 +-
 .../ql/udf/generic/GenericUDFOPGreaterThan.java |8 +-
 .../ql/udf/generic/GenericUDFOPLessThan.java|8 +-
 .../ql/udf/generic/GenericUDFOPNotEqual.java|8 +-
 .../ql/udf/generic/GenericUDFTimestamp.java |3 +-
 .../exec/vector/TestVectorizationContext.java   |4 +-
 .../expressions/TestVectorArithmetic.java   |   11 +
 .../expressions/TestVectorCastStatement.java|   42 +-
 .../expressions/TestVectorDateAddSub.java   |   10 +
 .../vector/expressions/TestVectorDateDiff.java  |   11 +
 .../expressions/TestVectorFilterCompare.java|  795 
 .../expressions/TestVectorIfStatement.java  |   74 +-
 .../vector/expressions/TestVectorNegative.java  |9 +
 .../expressions/TestVectorStringConcat.java |   11 +
 .../expressions/TestVectorStringUnary.java  |   12 +-
 .../vector/expressions/TestVectorSubStr.java|9 +
 .../expressions/TestVectorTimestampExtract.java |9 +
 .../convert_decimal64_to_decimal.q.out  |6 +-
 .../llap/convert_decimal64_to_decimal.q.out |   10 +-
 .../llap/vector_case_when_1.q.out   |   12 +-
 .../llap/vector_decimal_mapjoin.q.ou

[05/11] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 27d6c1c..b66fb9f 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
@@ -740,7 +740,7 @@ STAGE PLANS:
 Filter Vectorization:
 className: VectorFilterOperator
 native: true
-predicateExpression: SelectColumnIsNotNull(col 
3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 
0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+predicateExpression: SelectColumnIsNotNull(col 
0:decimal(14,2)/DECIMAL_64)
 predicate: dec is not null (type: boolean)
 Statistics: Num rows: 997 Data size: 106235 Basic stats: 
COMPLETE Column stats: NONE
 Select Operator
@@ -758,8 +758,8 @@ STAGE PLANS:
   0 _col0 (type: decimal(16,2))
   1 _col0 (type: decimal(16,2))
 Map Join Vectorization:
-bigTableKeyExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-bigTableValueExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2)
+bigTableKeyExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 3:decimal(16,2)
+bigTableValueExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
 className: VectorMapJoinOperator
 native: false
 nativeConditionsMet: 
hive.mapjoin.optimized.hashtable IS true, 
hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine 
tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS 
true, Small table vectorizes IS true
@@ -795,7 +795,7 @@ STAGE PLANS:
 includeColumns: [0]
 dataColumns: dec:decimal(14,2)/DECIMAL_64, 
value_dec:decimal(14,2)/DECIMAL_64
 partitionColumnCount: 0
-scratchColumnTypeNames: [decimal(14,2), decimal(16,2), 
decimal(16,2)]
+scratchColumnTypeNames: [decimal(16,2), decimal(16,2)]
 Map 2 
 Map Operator Tree:
 TableScan
@@ -809,7 +809,7 @@ STAGE PLANS:
 Filter Vectorization:
 className: VectorFilterOperator
 native: true
-predicateExpression: SelectColumnIsNotNull(col 
3:decimal(14,0))(children: ConvertDecimal64ToDecimal(col 
0:decimal(14,0)/DECIMAL_64) -> 3:decimal(14,0))
+predicateExpression: SelectColumnIsNotNull(col 
0:decimal(14,0)/DECIMAL_64)
 predicate: dec is not null (type: boolean)
 Statistics: Num rows: 997 Data size: 106235 Basic stats: 
COMPLETE Column stats: NONE
 Select Operator
@@ -847,7 +847,7 @@ STAGE PLANS:
 includeColumns: [0]
 dataColumns: dec:decimal(14,0)/DECIMAL_64, 
value_dec:decimal(14,0)/DECIMAL_64
 partitionColumnCount: 0
-scratchColumnTypeNames: [decimal(14,0)]
+scratchColumnTypeNames: []
 
   Stage: Stage-0
 Fetch Operator
@@ -1017,7 +1017,7 @@ STAGE PLANS:
 Filter Vectorization:
 className: VectorFilterOperator
 native: true
-predicateExpression: SelectColumnIsNotNull(col 
3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 
0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+predicateExpression: SelectColumnIsNotNull(col 
0:decimal(14,2)/DECIMAL_64)
 predicate: dec is not null (type: boolean)
 Statistics: Num rows: 997 Data size: 212470 Basic stats: 
COMPLETE Column stats: NONE
 Select Operator
@@ -1035,8 +1035,8 @@ STAGE PLANS:
   0 _col0 (type: decimal(16,2))
   1 _col0 (type: decimal(16,2))
 Map Join Vectorization:
-bigTableKeyExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-bigTableValueExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2), 
ConvertDecimal64ToDecimal(col 1:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2)

[07/11] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
deleted file mode 100644
index c7efe84..000
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector.expressions;
-
-import java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColLessLongScalar extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongColLessLongScalar(int colNum, long value, int outputColumnNum) {
-super(outputColumnNum);
-this.colNum = colNum;
-this.value = value;
-  }
-
-  public LongColLessLongScalar() {
-super();
-
-// Dummy final assignments.
-colNum = -1;
-value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-if (childExpressions != null) {
-  super.evaluateChildren(batch);
-}
-
-LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-LongColumnVector outputColVector = (LongColumnVector) 
batch.cols[outputColumnNum];
-int[] sel = batch.selected;
-boolean[] inputIsNull = inputColVector.isNull;
-boolean[] outputIsNull = outputColVector.isNull;
-int n = batch.size;
-long[] vector = inputColVector.vector;
-long[] outputVector = outputColVector.vector;
-
-// return immediately if batch is empty
-if (n == 0) {
-  return;
-}
-
-// We do not need to do a column reset since we are carefully changing the 
output.
-outputColVector.isRepeating = false;
-
-if (inputColVector.isRepeating) {
-  if (inputColVector.noNulls || !inputIsNull[0]) {
-outputIsNull[0] = false;
-outputVector[0] = vector[0] < value ? 1 : 0;
-  } else {
-outputIsNull[0] = true;
-outputColVector.noNulls = false;
-  }
-  outputColVector.isRepeating = true;
-  return;
-}
-
-if (inputColVector.noNulls) {
-  if (batch.selectedInUse) {
-
-// CONSIDER: For large n, fill n or all of isNull array and use the 
tighter ELSE loop.
-
-if (!outputColVector.noNulls) {
-  for(int j = 0; j != n; j++) {
-   final int i = sel[j];
-   // Set isNull before call in case it changes it mind.
-   outputIsNull[i] = false;
-   outputVector[i] = (vector[i] - value) >>> 63;
- }
-} else {
-  for(int j = 0; j != n; j++) {
-final int i = sel[j];
-outputVector[i] = (vector[i] - value) >>> 63;
-  }
-}
-  } else {
-if (!outputColVector.noNulls) {
-
-  // Assume it is almost always a performance win to fill all of 
isNull so we can
-  // safely reset noNulls.
-  Arrays.fill(outputIsNull, false);
-  outputColVector.noNulls = true;
-}
-for(int i = 0; i != n; i++) {
-  outputVector[i] = (vector[i] - value) >>> 63;
-}
-  }
-} else /* there are nulls in the inputColVector */ {
-
-  // Carefully handle NULLs...
-
-  /*
-   * For better performance on LONG/DOUBLE we don't want the conditional
-   * statements inside the for loop.
-   */
-  outputColVector.noNulls = false;
-
-  if (batch.selectedInUse) {
-for(int j=0; j != n; j++) {
-  int i = sel[j];
-  outputIsNull[i] = inputIsNull[i];
-  outputVector[i] = (vector[i] - value) >>> 63;
-}
-  } else {
-System.arraycopy(inputIsNull

[11/11] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0725

2018-07-25 Thread sershe
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0725


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

Branch: refs/heads/master-txnstats
Commit: 758ff449099065a84c46d63f9418201c8a6731b1
Parents: 5e7a8b5 71c4987
Author: sergey 
Authored: Wed Jul 25 11:18:51 2018 -0700
Committer: sergey 
Committed: Wed Jul 25 11:18:51 2018 -0700

--
 .../VectorizedComparisonBench.java  |   19 +-
 .../tezplugins/LlapTaskSchedulerService.java|   12 +-
 .../ExpressionTemplates/ColumnCompareColumn.txt |4 +-
 .../ExpressionTemplates/ColumnCompareScalar.txt |4 +-
 .../Decimal64ColumnCompareDecimal64Column.txt   |   54 +
 .../Decimal64ColumnCompareDecimal64Scalar.txt   |   66 +
 .../Decimal64ScalarCompareDecimal64Column.txt   |   66 +
 .../DecimalColumnCompareDecimalColumn.txt   |  153 +
 .../DecimalColumnCompareDecimalScalar.txt   |  177 +
 .../DecimalScalarCompareDecimalColumn.txt   |  180 +
 .../ExpressionTemplates/IfExprColumnScalar.txt  |6 +-
 .../IfExprObjectColumnColumn.txt|   41 +
 .../IfExprObjectColumnScalar.txt|   22 +
 .../IfExprObjectScalarColumn.txt|   22 +
 .../ExpressionTemplates/IfExprScalarColumn.txt  |6 +-
 .../ExpressionTemplates/IfExprScalarScalar.txt  |6 +-
 .../ExpressionTemplates/ScalarCompareColumn.txt |4 +-
 .../exec/vector/VectorExpressionDescriptor.java |2 +
 .../ql/exec/vector/VectorizationContext.java|  222 +-
 .../exec/vector/VectorizationContext.java.orig  | 3771 ++
 .../vector/expressions/CastCharToBinary.java|   55 +
 .../expressions/CastStringToTimestamp.java  |  177 +
 .../IfExprCharScalarStringScalar.java   |2 +-
 .../IfExprDecimal64ColumnDecimal64Column.java   |   55 +
 .../IfExprDecimal64ColumnDecimal64Scalar.java   |   70 +
 .../IfExprDecimal64ScalarDecimal64Column.java   |   71 +
 .../IfExprDecimal64ScalarDecimal64Scalar.java   |   75 +
 .../expressions/IfExprLongColumnLongColumn.java |6 +-
 ...fExprStringGroupColumnStringGroupColumn.java |4 +-
 .../IfExprStringGroupColumnStringScalar.java|4 +-
 .../IfExprStringScalarStringGroupColumn.java|4 +-
 .../IfExprStringScalarStringScalar.java |4 +-
 .../IfExprVarCharScalarStringScalar.java|2 +-
 .../expressions/LongColEqualLongColumn.java |  146 -
 .../expressions/LongColEqualLongScalar.java |  157 -
 .../LongColGreaterEqualLongColumn.java  |  146 -
 .../LongColGreaterEqualLongScalar.java  |  158 -
 .../expressions/LongColGreaterLongColumn.java   |  146 -
 .../expressions/LongColGreaterLongScalar.java   |  157 -
 .../expressions/LongColLessEqualLongColumn.java |  146 -
 .../expressions/LongColLessEqualLongScalar.java |  158 -
 .../expressions/LongColLessLongColumn.java  |  146 -
 .../expressions/LongColLessLongScalar.java  |  158 -
 .../expressions/LongColNotEqualLongColumn.java  |  146 -
 .../expressions/LongColNotEqualLongScalar.java  |  158 -
 .../expressions/LongScalarEqualLongColumn.java  |  157 -
 .../LongScalarGreaterEqualLongColumn.java   |  160 -
 .../LongScalarGreaterLongColumn.java|  161 -
 .../LongScalarLessEqualLongColumn.java  |  160 -
 .../expressions/LongScalarLessLongColumn.java   |  161 -
 .../LongScalarNotEqualLongColumn.java   |  161 -
 .../hive/ql/udf/generic/GenericUDFIf.java   |   16 +
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |8 +-
 .../generic/GenericUDFOPEqualOrGreaterThan.java |8 +-
 .../generic/GenericUDFOPEqualOrLessThan.java|8 +-
 .../ql/udf/generic/GenericUDFOPGreaterThan.java |8 +-
 .../ql/udf/generic/GenericUDFOPLessThan.java|8 +-
 .../ql/udf/generic/GenericUDFOPNotEqual.java|8 +-
 .../ql/udf/generic/GenericUDFTimestamp.java |3 +-
 .../exec/vector/TestVectorizationContext.java   |4 +-
 .../expressions/TestVectorArithmetic.java   |   11 +
 .../expressions/TestVectorCastStatement.java|   42 +-
 .../expressions/TestVectorDateAddSub.java   |   10 +
 .../vector/expressions/TestVectorDateDiff.java  |   11 +
 .../expressions/TestVectorFilterCompare.java|  795 
 .../expressions/TestVectorIfStatement.java  |   74 +-
 .../vector/expressions/TestVectorNegative.java  |9 +
 .../expressions/TestVectorStringConcat.java |   11 +
 .../expressions/TestVectorStringUnary.java  |   12 +-
 .../vector/expressions/TestVectorSubStr.java|9 +
 .../expressions/TestVectorTimestampExtract.java |9 +
 .../clientnegative/alter_table_wrong_db.q   |4 +
 .../clientnegative/alter_table_wrong_table.q|3 +
 .../clientnegative/alter_table_wrong_db.q.out   |   25 +
 .../alter_table_wrong_table.q.out

[01/11] hive git commit: HIVE-19935 : Hive WM session killed: Failed to update LLAP tasks count (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

2018-07-25 Thread sershe
Repository: hive
Updated Branches:
  refs/heads/master-txnstats 5e7a8b59c -> 758ff4490


HIVE-19935 : Hive WM session killed: Failed to update LLAP tasks count (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/91308116
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/91308116
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/91308116

Branch: refs/heads/master-txnstats
Commit: 913081169730e86969974023916df804d7b2a7e0
Parents: 26f4d8e
Author: sergey 
Authored: Tue Jul 24 15:15:05 2018 -0700
Committer: sergey 
Committed: Tue Jul 24 15:15:05 2018 -0700

--
 .../hive/llap/tezplugins/LlapTaskSchedulerService.java  | 12 
 1 file changed, 8 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/91308116/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
--
diff --git 
a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
 
b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
index 8217964..b748c7e 100644
--- 
a/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
+++ 
b/llap-tez/src/java/org/apache/hadoop/hive/llap/tezplugins/LlapTaskSchedulerService.java
@@ -587,11 +587,14 @@ public class LlapTaskSchedulerService extends 
TaskScheduler {
 boolean newState = false;
 synchronized (ti) {
   assert ti.isPendingUpdate;
-  if (ti.lastSetGuaranteed != null && ti.lastSetGuaranteed == 
ti.isGuaranteed) {
+  if ((ti.lastSetGuaranteed != null && ti.lastSetGuaranteed == 
ti.isGuaranteed)
+  || ti.isGuaranteed == null) {
+// Nothing to do - e.g. two messages have canceled each other before 
we could react,
+// or the task was deallocated.
 ti.requestedValue = ti.isGuaranteed;
 setUpdateDoneUnderTiLock(ti);
 WM_LOG.info("Not sending update to " + ti.attemptId);
-return; // Nothing to do - e.g. two messages have canceled each other 
before we could react.
+return;
   }
   newState = ti.isGuaranteed;
 }
@@ -612,7 +615,8 @@ public class LlapTaskSchedulerService extends TaskScheduler 
{
   private void setUpdateDoneUnderTiLock(TaskInfo ti) {
 ti.isPendingUpdate = false;
 // It's ok to update metrics for two tasks in parallel, but not for the 
same one.
-if (metrics != null) {
+// Don't update metrics for the cancelled tasks - already taken care of 
during cancellation.
+if (metrics != null && ti.requestedValue != null) {
   metrics.setWmPendingDone(ti.requestedValue);
 }
 ti.lastSetGuaranteed = ti.requestedValue;
@@ -660,7 +664,7 @@ public class LlapTaskSchedulerService extends TaskScheduler 
{
 // update the pending state for now as we release this lock to take 
both.
 newStateAnyTask = requestedValue;
   }
-} // End of synchronized (ti) 
+} // End of synchronized (ti)
 if (newStateSameTask != null) {
   WM_LOG.info("Sending update to the same task in response handling "
   + ti.attemptId + ", " + newStateSameTask);



[03/11] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 8b36371..666572a 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
@@ -316,16 +316,22 @@ public class GenVectorCode extends Task {
   {"Decimal64ColumnArithmeticDecimal64Column", "Add", "+"},
   {"Decimal64ColumnArithmeticDecimal64Column", "Subtract", "-"},
 
+  {"ColumnCompareScalar", "Equal", "long", "long", "=="},
   {"ColumnCompareScalar", "Equal", "long", "double", "=="},
   {"ColumnCompareScalar", "Equal", "double", "double", "=="},
+  {"ColumnCompareScalar", "NotEqual", "long", "long", "!="},
   {"ColumnCompareScalar", "NotEqual", "long", "double", "!="},
   {"ColumnCompareScalar", "NotEqual", "double", "double", "!="},
+  {"ColumnCompareScalar", "Less", "long", "long", "<"},
   {"ColumnCompareScalar", "Less", "long", "double", "<"},
   {"ColumnCompareScalar", "Less", "double", "double", "<"},
+  {"ColumnCompareScalar", "LessEqual", "long", "long", "<="},
   {"ColumnCompareScalar", "LessEqual", "long", "double", "<="},
   {"ColumnCompareScalar", "LessEqual", "double", "double", "<="},
+  {"ColumnCompareScalar", "Greater", "long", "long", ">"},
   {"ColumnCompareScalar", "Greater", "long", "double", ">"},
   {"ColumnCompareScalar", "Greater", "double", "double", ">"},
+  {"ColumnCompareScalar", "GreaterEqual", "long", "long", ">="},
   {"ColumnCompareScalar", "GreaterEqual", "long", "double", ">="},
   {"ColumnCompareScalar", "GreaterEqual", "double", "double", ">="},
 
@@ -336,16 +342,22 @@ public class GenVectorCode extends Task {
   {"ColumnCompareScalar", "Greater", "double", "long", ">"},
   {"ColumnCompareScalar", "GreaterEqual", "double", "long", ">="},
 
+  {"ScalarCompareColumn", "Equal", "long", "long", "=="},
   {"ScalarCompareColumn", "Equal", "long", "double", "=="},
   {"ScalarCompareColumn", "Equal", "double", "double", "=="},
+  {"ScalarCompareColumn", "NotEqual", "long", "long", "!="},
   {"ScalarCompareColumn", "NotEqual", "long", "double", "!="},
   {"ScalarCompareColumn", "NotEqual", "double", "double", "!="},
+  {"ScalarCompareColumn", "Less", "long", "long", "<"},
   {"ScalarCompareColumn", "Less", "long", "double", "<"},
   {"ScalarCompareColumn", "Less", "double", "double", "<"},
+  {"ScalarCompareColumn", "LessEqual", "long", "long", "<="},
   {"ScalarCompareColumn", "LessEqual", "long", "double", "<="},
   {"ScalarCompareColumn", "LessEqual", "double", "double", "<="},
+  {"ScalarCompareColumn", "Greater", "long", "long", ">"},
   {"ScalarCompareColumn", "Greater", "long", "double", ">"},
   {"ScalarCompareColumn", "Greater", "double", "double", ">"},
+  {"ScalarCompareColumn", "GreaterEqual", "long", "long", ">="},
   {"ScalarCompareColumn", "GreaterEqual", "long", "double", ">="},
   {"ScalarCompareColumn", "GreaterEqual", "double", "double", ">="},
 
@@ -356,6 +368,28 @@ public class GenVectorCode extends Task {
   {"ScalarCompareColumn", "Greater", "double", "long", ">"},
   {"ScalarCompareColumn", "GreaterEqual", "double", "long", ">="},
 
+  // Compare decimal to decimal.
+  {"DecimalCompareDecimal", "Equal", "==", "Col", "Column"},
+  {"DecimalCompareDecimal", "NotEqual", "!=", "Col", "Column"},
+  {"DecimalCompareDecimal", "Less", "<", "Col", "Column"},
+  {"DecimalCompareDecimal", "LessEqual", "<=", "Col", "Column"},
+  {"DecimalCompareDecimal", "Greater", ">", "Col", "Column"},
+  {"DecimalCompareDecimal", "GreaterEqual", ">=", "Col", "Column"},
+
+  {"DecimalCompareDecimal", "Equal", "==", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "NotEqual", "!=", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "Less", "<", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "LessEqual", "<=", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "Greater", ">", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "GreaterEqual", ">=", "Col", "Scalar"},
+
+  {"DecimalCompareDecimal", "Equal", "==", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "NotEqual", "!=", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "Less", "<", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "LessEqual", "<=", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "Greater", ">", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "GreaterEqual", ">=", "Scalar", "Column"},
+
   // Compare timestamp to timestamp.
   {"TimestampCompareTimestamp", "Equal", "==", "timestamp", "Col", 
"Column"},
   {"TimestampCompareT

[02/11] hive git commit: HIVE-18852: Misleading error message in alter table validation (Andrew Sherman, reviewed by Vineet Garg)

2018-07-25 Thread sershe
HIVE-18852: Misleading error message in alter table validation (Andrew Sherman, 
reviewed by Vineet Garg)


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

Branch: refs/heads/master-txnstats
Commit: ca0d70697f3cbb20fbba4c1c56a3f3253fe919dd
Parents: 9130811
Author: Andrew Sherman 
Authored: Tue Jul 24 19:57:12 2018 -0700
Committer: Vineet Garg 
Committed: Tue Jul 24 19:59:40 2018 -0700

--
 .../clientnegative/alter_table_wrong_db.q   |  4 
 .../clientnegative/alter_table_wrong_table.q|  3 +++
 .../clientnegative/alter_table_wrong_db.q.out   | 25 
 .../alter_table_wrong_table.q.out   | 13 ++
 .../hadoop/hive/metastore/HiveAlterHandler.java |  4 ++--
 5 files changed, 47 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/ca0d7069/ql/src/test/queries/clientnegative/alter_table_wrong_db.q
--
diff --git a/ql/src/test/queries/clientnegative/alter_table_wrong_db.q 
b/ql/src/test/queries/clientnegative/alter_table_wrong_db.q
new file mode 100644
index 000..da8ee48
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/alter_table_wrong_db.q
@@ -0,0 +1,4 @@
+create database bad_rename1;
+use bad_rename1;
+create table rename1(a int);
+alter table bad_rename1.rename1 rename to bad_db_notexists.rename1;

http://git-wip-us.apache.org/repos/asf/hive/blob/ca0d7069/ql/src/test/queries/clientnegative/alter_table_wrong_table.q
--
diff --git a/ql/src/test/queries/clientnegative/alter_table_wrong_table.q 
b/ql/src/test/queries/clientnegative/alter_table_wrong_table.q
new file mode 100644
index 000..229656e
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/alter_table_wrong_table.q
@@ -0,0 +1,3 @@
+create database bad_rename2;
+use bad_rename2;
+alter table bad_rename2.rename_not_exists rename to bad_db_notexists.rename1;

http://git-wip-us.apache.org/repos/asf/hive/blob/ca0d7069/ql/src/test/results/clientnegative/alter_table_wrong_db.q.out
--
diff --git a/ql/src/test/results/clientnegative/alter_table_wrong_db.q.out 
b/ql/src/test/results/clientnegative/alter_table_wrong_db.q.out
new file mode 100644
index 000..641e09f
--- /dev/null
+++ b/ql/src/test/results/clientnegative/alter_table_wrong_db.q.out
@@ -0,0 +1,25 @@
+PREHOOK: query: create database bad_rename1
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:bad_rename1
+POSTHOOK: query: create database bad_rename1
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:bad_rename1
+PREHOOK: query: use bad_rename1
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:bad_rename1
+POSTHOOK: query: use bad_rename1
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:bad_rename1
+PREHOOK: query: create table rename1(a int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: bad_rename1@rename1
+PREHOOK: Output: database:bad_rename1
+POSTHOOK: query: create table rename1(a int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: bad_rename1@rename1
+POSTHOOK: Output: database:bad_rename1
+PREHOOK: query: alter table bad_rename1.rename1 rename to 
bad_db_notexists.rename1
+PREHOOK: type: ALTERTABLE_RENAME
+PREHOOK: Input: bad_rename1@rename1
+PREHOOK: Output: bad_rename1@rename1
+FAILED: Execution Error, return code 1 from 
org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. Unable to change 
partition or table. Object bad_db_notexists does not exist. Check metastore 
logs for detailed stack.

http://git-wip-us.apache.org/repos/asf/hive/blob/ca0d7069/ql/src/test/results/clientnegative/alter_table_wrong_table.q.out
--
diff --git a/ql/src/test/results/clientnegative/alter_table_wrong_table.q.out 
b/ql/src/test/results/clientnegative/alter_table_wrong_table.q.out
new file mode 100644
index 000..35087f8
--- /dev/null
+++ b/ql/src/test/results/clientnegative/alter_table_wrong_table.q.out
@@ -0,0 +1,13 @@
+PREHOOK: query: create database bad_rename2
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:bad_rename2
+POSTHOOK: query: create database bad_rename2
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:bad_rename2
+PREHOOK: query: use bad_rename2
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:bad_rename2
+POSTHOOK: query: use bad_rename2
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:bad_rename2
+FAILED: SemanticException [Error 10001]: Table not found 
bad_rename2.rename_not_exists

http://git-wip-us.apache.org/rep

[04/11] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
--
diff --git 
a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out 
b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
index 1791c89..f094fba 100644
--- a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
@@ -476,7 +476,7 @@ STAGE PLANS:
 className: VectorSelectOperator
 native: true
 projectedOutputColumnNums: [5, 7, 8, 9, 6, 11, 10, 13, 
14]
-selectExpressions: VectorUDFUnixTimeStampString(col 
2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: 
CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, 
field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, 
VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: 
CastStringToDate(col 2:string) -> 6:date) -> 9:int, 
VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, 
VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: 
CastStringToDate(col 2:string) -> 10:date) -> 11:int, 
VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: 
VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 10:int, 
VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: 
VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 13:int, 
VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: 
VectorUDFAdaptor(CAST( stimest
 amp1 AS TIMESTAMP)) -> 12:timestamp) -> 14:int
+selectExpressions: VectorUDFUnixTimeStampString(col 
2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: 
CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, 
field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, 
VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: 
CastStringToDate(col 2:string) -> 6:date) -> 9:int, 
VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, 
VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: 
CastStringToDate(col 2:string) -> 10:date) -> 11:int, 
VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: 
CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 10:int, 
VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: 
CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 13:int, 
VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: 
CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 14
 :int
 Statistics: Num rows: 52 Data size: 3219 Basic stats: 
COMPLETE Column stats: NONE
 Reduce Output Operator
   key expressions: _col0 (type: bigint)
@@ -495,7 +495,7 @@ STAGE PLANS:
 featureSupportInUse: [DECIMAL_64]
 inputFileFormats: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
 allNative: true
-usesVectorUDFAdaptor: true
+usesVectorUDFAdaptor: false
 vectorized: true
 Reducer 2 
 Execution mode: vectorized
@@ -668,7 +668,7 @@ STAGE PLANS:
 className: VectorSelectOperator
 native: true
 projectedOutputColumnNums: [7, 6, 8, 9, 11, 10, 14, 
15, 16]
-selectExpressions: LongColEqualLongColumn(col 
5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 
1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 
6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 8:int)(children: 
VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, 
VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) 
-> 6:date) -> 8:int) -> 6:boolean, LongColEqualLongColumn(col 5:int, col 
9:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 
5:int, VectorUDFMonthDate(col 8, field MONTH)(children: CastStringToDate(col 
2:string) -> 8:date) -> 9:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, 
col 10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field 
DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthDate(col 9, field 
DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 9:date) -> 10:int) -> 
9:boolean, LongColEqualLongColumn(col 5:int, col
  10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field 
DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, 
fieldLength 2) -> 10:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 
12:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field 
WEEK_OF_YEAR) -> 5:int, Ve

[09/11] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
new file mode 100644
index 000..20cc894
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
@@ -0,0 +1,3771 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector;
+
+import java.lang.reflect.Constructor;
+import java.nio.charset.StandardCharsets;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.hive.common.type.Date;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
+import 
org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.ArgumentType;
+import 
org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.InputExpressionType;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFArgDesc;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+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.plan.ExprNodeFieldDesc;
+import org.apache.hadoop.hive.ql.udf.*;
+import org.apache.hadoop.hive.ql.udf.generic.*;
+import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import 
org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache

[08/11] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
new file mode 100644
index 000..f8f60a8
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
+
+public class CastCharToBinary extends StringUnaryUDFDirect {
+
+  private static final long serialVersionUID = 1L;
+  private int maxLength;
+
+  public CastCharToBinary(int inputColumn, int outputColumnNum) {
+super(inputColumn, outputColumnNum);
+  }
+
+  public CastCharToBinary() {
+super();
+  }
+
+  @Override
+  public void transientInit() throws HiveException {
+super.transientInit();
+
+maxLength = ((CharTypeInfo) inputTypeInfos[0]).getLength();
+  }
+
+  /**
+   * Do pad out the CHAR type into the BINARY result, taking into account 
Unicode...
+   */
+  protected void func(BytesColumnVector outV, byte[][] vector, int[] start, 
int[] length, int i) {
+StringExpr.padRight(outV, i, vector[i], start[i], length[i], maxLength);
+  }
+
+  public String vectorExpressionParameters() {
+return getColumnParamString(0, inputColumn) + ", maxLength " + maxLength;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
new file mode 100644
index 000..b48b013
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import java.util.Arrays;
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+
+
+/**
+ * Casts a string vector to a Timestamp vector.
+ */
+public class CastStringToTimestamp extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+
+  private final int inputColumn;
+
+  public CastStringToTimestamp() {
+super();
+
+// Dummy final assignments.
+inputColumn = -1;
+  }
+
+  public CastStringToTimestamp(int inputColumn, int outputColumnNum) {
+super(outputColumnNum);
+this.inputColumn = inputColumn;
+  }
+
+  @Override
+  public v

[06/11] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread sershe
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
--
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
new file mode 100644
index 000..1ff11ec
--- /dev/null
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
@@ -0,0 +1,795 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import java.lang.reflect.Constructor;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomBatchSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import 
org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+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.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateAdd;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateDiff;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateSub;
+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.GenericUDFOPEqualOrGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual;
+import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import 
org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
+import 
org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.had

[3/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread mmccline
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 27d6c1c..b66fb9f 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
@@ -740,7 +740,7 @@ STAGE PLANS:
 Filter Vectorization:
 className: VectorFilterOperator
 native: true
-predicateExpression: SelectColumnIsNotNull(col 
3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 
0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+predicateExpression: SelectColumnIsNotNull(col 
0:decimal(14,2)/DECIMAL_64)
 predicate: dec is not null (type: boolean)
 Statistics: Num rows: 997 Data size: 106235 Basic stats: 
COMPLETE Column stats: NONE
 Select Operator
@@ -758,8 +758,8 @@ STAGE PLANS:
   0 _col0 (type: decimal(16,2))
   1 _col0 (type: decimal(16,2))
 Map Join Vectorization:
-bigTableKeyExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-bigTableValueExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2)
+bigTableKeyExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 3:decimal(16,2)
+bigTableValueExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
 className: VectorMapJoinOperator
 native: false
 nativeConditionsMet: 
hive.mapjoin.optimized.hashtable IS true, 
hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine 
tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS 
true, Small table vectorizes IS true
@@ -795,7 +795,7 @@ STAGE PLANS:
 includeColumns: [0]
 dataColumns: dec:decimal(14,2)/DECIMAL_64, 
value_dec:decimal(14,2)/DECIMAL_64
 partitionColumnCount: 0
-scratchColumnTypeNames: [decimal(14,2), decimal(16,2), 
decimal(16,2)]
+scratchColumnTypeNames: [decimal(16,2), decimal(16,2)]
 Map 2 
 Map Operator Tree:
 TableScan
@@ -809,7 +809,7 @@ STAGE PLANS:
 Filter Vectorization:
 className: VectorFilterOperator
 native: true
-predicateExpression: SelectColumnIsNotNull(col 
3:decimal(14,0))(children: ConvertDecimal64ToDecimal(col 
0:decimal(14,0)/DECIMAL_64) -> 3:decimal(14,0))
+predicateExpression: SelectColumnIsNotNull(col 
0:decimal(14,0)/DECIMAL_64)
 predicate: dec is not null (type: boolean)
 Statistics: Num rows: 997 Data size: 106235 Basic stats: 
COMPLETE Column stats: NONE
 Select Operator
@@ -847,7 +847,7 @@ STAGE PLANS:
 includeColumns: [0]
 dataColumns: dec:decimal(14,0)/DECIMAL_64, 
value_dec:decimal(14,0)/DECIMAL_64
 partitionColumnCount: 0
-scratchColumnTypeNames: [decimal(14,0)]
+scratchColumnTypeNames: []
 
   Stage: Stage-0
 Fetch Operator
@@ -1017,7 +1017,7 @@ STAGE PLANS:
 Filter Vectorization:
 className: VectorFilterOperator
 native: true
-predicateExpression: SelectColumnIsNotNull(col 
3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 
0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+predicateExpression: SelectColumnIsNotNull(col 
0:decimal(14,2)/DECIMAL_64)
 predicate: dec is not null (type: boolean)
 Statistics: Num rows: 997 Data size: 212470 Basic stats: 
COMPLETE Column stats: NONE
 Select Operator
@@ -1035,8 +1035,8 @@ STAGE PLANS:
   0 _col0 (type: decimal(16,2))
   1 _col0 (type: decimal(16,2))
 Map Join Vectorization:
-bigTableKeyExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-bigTableValueExpressions: 
ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2), 
ConvertDecimal64ToDecimal(col 1:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2)

[5/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread mmccline
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
deleted file mode 100644
index c7efe84..000
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector.expressions;
-
-import java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColLessLongScalar extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongColLessLongScalar(int colNum, long value, int outputColumnNum) {
-super(outputColumnNum);
-this.colNum = colNum;
-this.value = value;
-  }
-
-  public LongColLessLongScalar() {
-super();
-
-// Dummy final assignments.
-colNum = -1;
-value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-if (childExpressions != null) {
-  super.evaluateChildren(batch);
-}
-
-LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-LongColumnVector outputColVector = (LongColumnVector) 
batch.cols[outputColumnNum];
-int[] sel = batch.selected;
-boolean[] inputIsNull = inputColVector.isNull;
-boolean[] outputIsNull = outputColVector.isNull;
-int n = batch.size;
-long[] vector = inputColVector.vector;
-long[] outputVector = outputColVector.vector;
-
-// return immediately if batch is empty
-if (n == 0) {
-  return;
-}
-
-// We do not need to do a column reset since we are carefully changing the 
output.
-outputColVector.isRepeating = false;
-
-if (inputColVector.isRepeating) {
-  if (inputColVector.noNulls || !inputIsNull[0]) {
-outputIsNull[0] = false;
-outputVector[0] = vector[0] < value ? 1 : 0;
-  } else {
-outputIsNull[0] = true;
-outputColVector.noNulls = false;
-  }
-  outputColVector.isRepeating = true;
-  return;
-}
-
-if (inputColVector.noNulls) {
-  if (batch.selectedInUse) {
-
-// CONSIDER: For large n, fill n or all of isNull array and use the 
tighter ELSE loop.
-
-if (!outputColVector.noNulls) {
-  for(int j = 0; j != n; j++) {
-   final int i = sel[j];
-   // Set isNull before call in case it changes it mind.
-   outputIsNull[i] = false;
-   outputVector[i] = (vector[i] - value) >>> 63;
- }
-} else {
-  for(int j = 0; j != n; j++) {
-final int i = sel[j];
-outputVector[i] = (vector[i] - value) >>> 63;
-  }
-}
-  } else {
-if (!outputColVector.noNulls) {
-
-  // Assume it is almost always a performance win to fill all of 
isNull so we can
-  // safely reset noNulls.
-  Arrays.fill(outputIsNull, false);
-  outputColVector.noNulls = true;
-}
-for(int i = 0; i != n; i++) {
-  outputVector[i] = (vector[i] - value) >>> 63;
-}
-  }
-} else /* there are nulls in the inputColVector */ {
-
-  // Carefully handle NULLs...
-
-  /*
-   * For better performance on LONG/DOUBLE we don't want the conditional
-   * statements inside the for loop.
-   */
-  outputColVector.noNulls = false;
-
-  if (batch.selectedInUse) {
-for(int j=0; j != n; j++) {
-  int i = sel[j];
-  outputIsNull[i] = inputIsNull[i];
-  outputVector[i] = (vector[i] - value) >>> 63;
-}
-  } else {
-System.arraycopy(inputIsNull

[7/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread mmccline
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
new file mode 100644
index 000..20cc894
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
@@ -0,0 +1,3771 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector;
+
+import java.lang.reflect.Constructor;
+import java.nio.charset.StandardCharsets;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.hive.common.type.Date;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
+import 
org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.ArgumentType;
+import 
org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.InputExpressionType;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFArgDesc;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+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.plan.ExprNodeFieldDesc;
+import org.apache.hadoop.hive.ql.udf.*;
+import org.apache.hadoop.hive.ql.udf.generic.*;
+import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import 
org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import 
org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache

[1/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread mmccline
Repository: hive
Updated Branches:
  refs/heads/master ca0d70697 -> 71c49878c


http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 8b36371..666572a 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
@@ -316,16 +316,22 @@ public class GenVectorCode extends Task {
   {"Decimal64ColumnArithmeticDecimal64Column", "Add", "+"},
   {"Decimal64ColumnArithmeticDecimal64Column", "Subtract", "-"},
 
+  {"ColumnCompareScalar", "Equal", "long", "long", "=="},
   {"ColumnCompareScalar", "Equal", "long", "double", "=="},
   {"ColumnCompareScalar", "Equal", "double", "double", "=="},
+  {"ColumnCompareScalar", "NotEqual", "long", "long", "!="},
   {"ColumnCompareScalar", "NotEqual", "long", "double", "!="},
   {"ColumnCompareScalar", "NotEqual", "double", "double", "!="},
+  {"ColumnCompareScalar", "Less", "long", "long", "<"},
   {"ColumnCompareScalar", "Less", "long", "double", "<"},
   {"ColumnCompareScalar", "Less", "double", "double", "<"},
+  {"ColumnCompareScalar", "LessEqual", "long", "long", "<="},
   {"ColumnCompareScalar", "LessEqual", "long", "double", "<="},
   {"ColumnCompareScalar", "LessEqual", "double", "double", "<="},
+  {"ColumnCompareScalar", "Greater", "long", "long", ">"},
   {"ColumnCompareScalar", "Greater", "long", "double", ">"},
   {"ColumnCompareScalar", "Greater", "double", "double", ">"},
+  {"ColumnCompareScalar", "GreaterEqual", "long", "long", ">="},
   {"ColumnCompareScalar", "GreaterEqual", "long", "double", ">="},
   {"ColumnCompareScalar", "GreaterEqual", "double", "double", ">="},
 
@@ -336,16 +342,22 @@ public class GenVectorCode extends Task {
   {"ColumnCompareScalar", "Greater", "double", "long", ">"},
   {"ColumnCompareScalar", "GreaterEqual", "double", "long", ">="},
 
+  {"ScalarCompareColumn", "Equal", "long", "long", "=="},
   {"ScalarCompareColumn", "Equal", "long", "double", "=="},
   {"ScalarCompareColumn", "Equal", "double", "double", "=="},
+  {"ScalarCompareColumn", "NotEqual", "long", "long", "!="},
   {"ScalarCompareColumn", "NotEqual", "long", "double", "!="},
   {"ScalarCompareColumn", "NotEqual", "double", "double", "!="},
+  {"ScalarCompareColumn", "Less", "long", "long", "<"},
   {"ScalarCompareColumn", "Less", "long", "double", "<"},
   {"ScalarCompareColumn", "Less", "double", "double", "<"},
+  {"ScalarCompareColumn", "LessEqual", "long", "long", "<="},
   {"ScalarCompareColumn", "LessEqual", "long", "double", "<="},
   {"ScalarCompareColumn", "LessEqual", "double", "double", "<="},
+  {"ScalarCompareColumn", "Greater", "long", "long", ">"},
   {"ScalarCompareColumn", "Greater", "long", "double", ">"},
   {"ScalarCompareColumn", "Greater", "double", "double", ">"},
+  {"ScalarCompareColumn", "GreaterEqual", "long", "long", ">="},
   {"ScalarCompareColumn", "GreaterEqual", "long", "double", ">="},
   {"ScalarCompareColumn", "GreaterEqual", "double", "double", ">="},
 
@@ -356,6 +368,28 @@ public class GenVectorCode extends Task {
   {"ScalarCompareColumn", "Greater", "double", "long", ">"},
   {"ScalarCompareColumn", "GreaterEqual", "double", "long", ">="},
 
+  // Compare decimal to decimal.
+  {"DecimalCompareDecimal", "Equal", "==", "Col", "Column"},
+  {"DecimalCompareDecimal", "NotEqual", "!=", "Col", "Column"},
+  {"DecimalCompareDecimal", "Less", "<", "Col", "Column"},
+  {"DecimalCompareDecimal", "LessEqual", "<=", "Col", "Column"},
+  {"DecimalCompareDecimal", "Greater", ">", "Col", "Column"},
+  {"DecimalCompareDecimal", "GreaterEqual", ">=", "Col", "Column"},
+
+  {"DecimalCompareDecimal", "Equal", "==", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "NotEqual", "!=", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "Less", "<", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "LessEqual", "<=", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "Greater", ">", "Col", "Scalar"},
+  {"DecimalCompareDecimal", "GreaterEqual", ">=", "Col", "Scalar"},
+
+  {"DecimalCompareDecimal", "Equal", "==", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "NotEqual", "!=", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "Less", "<", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "LessEqual", "<=", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "Greater", ">", "Scalar", "Column"},
+  {"DecimalCompareDecimal", "GreaterEqual", ">=", "Scalar", "Column"},
+
   // Compare timestamp to timestamp.
   {"TimestampCompareTimes

[4/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread mmccline
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
--
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
new file mode 100644
index 000..1ff11ec
--- /dev/null
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
@@ -0,0 +1,795 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import java.lang.reflect.Constructor;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomBatchSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import 
org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+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.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateAdd;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateDiff;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateSub;
+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.GenericUDFOPEqualOrGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual;
+import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import 
org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
+import 
org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.had

[2/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread mmccline
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
--
diff --git 
a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out 
b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
index 1791c89..f094fba 100644
--- a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
@@ -476,7 +476,7 @@ STAGE PLANS:
 className: VectorSelectOperator
 native: true
 projectedOutputColumnNums: [5, 7, 8, 9, 6, 11, 10, 13, 
14]
-selectExpressions: VectorUDFUnixTimeStampString(col 
2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: 
CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, 
field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, 
VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: 
CastStringToDate(col 2:string) -> 6:date) -> 9:int, 
VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, 
VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: 
CastStringToDate(col 2:string) -> 10:date) -> 11:int, 
VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: 
VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 10:int, 
VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: 
VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 13:int, 
VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: 
VectorUDFAdaptor(CAST( stimest
 amp1 AS TIMESTAMP)) -> 12:timestamp) -> 14:int
+selectExpressions: VectorUDFUnixTimeStampString(col 
2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: 
CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, 
field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, 
VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: 
CastStringToDate(col 2:string) -> 6:date) -> 9:int, 
VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, 
VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: 
CastStringToDate(col 2:string) -> 10:date) -> 11:int, 
VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: 
CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 10:int, 
VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: 
CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 13:int, 
VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: 
CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 14
 :int
 Statistics: Num rows: 52 Data size: 3219 Basic stats: 
COMPLETE Column stats: NONE
 Reduce Output Operator
   key expressions: _col0 (type: bigint)
@@ -495,7 +495,7 @@ STAGE PLANS:
 featureSupportInUse: [DECIMAL_64]
 inputFileFormats: 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
 allNative: true
-usesVectorUDFAdaptor: true
+usesVectorUDFAdaptor: false
 vectorized: true
 Reducer 2 
 Execution mode: vectorized
@@ -668,7 +668,7 @@ STAGE PLANS:
 className: VectorSelectOperator
 native: true
 projectedOutputColumnNums: [7, 6, 8, 9, 11, 10, 14, 
15, 16]
-selectExpressions: LongColEqualLongColumn(col 
5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 
1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 
6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 8:int)(children: 
VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, 
VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) 
-> 6:date) -> 8:int) -> 6:boolean, LongColEqualLongColumn(col 5:int, col 
9:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 
5:int, VectorUDFMonthDate(col 8, field MONTH)(children: CastStringToDate(col 
2:string) -> 8:date) -> 9:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, 
col 10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field 
DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthDate(col 9, field 
DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 9:date) -> 10:int) -> 
9:boolean, LongColEqualLongColumn(col 5:int, col
  10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field 
DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, 
fieldLength 2) -> 10:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 
12:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field 
WEEK_OF_YEAR) -> 5:int, Ve

[6/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread mmccline
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
new file mode 100644
index 000..f8f60a8
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
+
+public class CastCharToBinary extends StringUnaryUDFDirect {
+
+  private static final long serialVersionUID = 1L;
+  private int maxLength;
+
+  public CastCharToBinary(int inputColumn, int outputColumnNum) {
+super(inputColumn, outputColumnNum);
+  }
+
+  public CastCharToBinary() {
+super();
+  }
+
+  @Override
+  public void transientInit() throws HiveException {
+super.transientInit();
+
+maxLength = ((CharTypeInfo) inputTypeInfos[0]).getLength();
+  }
+
+  /**
+   * Do pad out the CHAR type into the BINARY result, taking into account 
Unicode...
+   */
+  protected void func(BytesColumnVector outV, byte[][] vector, int[] start, 
int[] length, int i) {
+StringExpr.padRight(outV, i, vector[i], start[i], length[i], maxLength);
+  }
+
+  public String vectorExpressionParameters() {
+return getColumnParamString(0, inputColumn) + ", maxLength " + maxLength;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
new file mode 100644
index 000..b48b013
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import java.util.Arrays;
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+
+
+/**
+ * Casts a string vector to a Timestamp vector.
+ */
+public class CastStringToTimestamp extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+
+  private final int inputColumn;
+
+  public CastStringToTimestamp() {
+super();
+
+// Dummy final assignments.
+inputColumn = -1;
+  }
+
+  public CastStringToTimestamp(int inputColumn, int outputColumnNum) {
+super(outputColumnNum);
+this.inputColumn = inputColumn;
+  }
+
+  @Override
+  public v

[8/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

2018-07-25 Thread mmccline
HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare 
(Matt McCline, reviewed by Teddy Choi)


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

Branch: refs/heads/master
Commit: 71c49878c3669f19f900ec1ffe7652c91ce15d38
Parents: ca0d706
Author: Matt McCline 
Authored: Wed Jul 25 09:58:51 2018 -0500
Committer: Matt McCline 
Committed: Wed Jul 25 09:59:35 2018 -0500

--
 .../VectorizedComparisonBench.java  |   19 +-
 .../ExpressionTemplates/ColumnCompareColumn.txt |4 +-
 .../ExpressionTemplates/ColumnCompareScalar.txt |4 +-
 .../Decimal64ColumnCompareDecimal64Column.txt   |   54 +
 .../Decimal64ColumnCompareDecimal64Scalar.txt   |   66 +
 .../Decimal64ScalarCompareDecimal64Column.txt   |   66 +
 .../DecimalColumnCompareDecimalColumn.txt   |  153 +
 .../DecimalColumnCompareDecimalScalar.txt   |  177 +
 .../DecimalScalarCompareDecimalColumn.txt   |  180 +
 .../ExpressionTemplates/IfExprColumnScalar.txt  |6 +-
 .../IfExprObjectColumnColumn.txt|   41 +
 .../IfExprObjectColumnScalar.txt|   22 +
 .../IfExprObjectScalarColumn.txt|   22 +
 .../ExpressionTemplates/IfExprScalarColumn.txt  |6 +-
 .../ExpressionTemplates/IfExprScalarScalar.txt  |6 +-
 .../ExpressionTemplates/ScalarCompareColumn.txt |4 +-
 .../exec/vector/VectorExpressionDescriptor.java |2 +
 .../ql/exec/vector/VectorizationContext.java|  222 +-
 .../exec/vector/VectorizationContext.java.orig  | 3771 ++
 .../vector/expressions/CastCharToBinary.java|   55 +
 .../expressions/CastStringToTimestamp.java  |  177 +
 .../IfExprCharScalarStringScalar.java   |2 +-
 .../IfExprDecimal64ColumnDecimal64Column.java   |   55 +
 .../IfExprDecimal64ColumnDecimal64Scalar.java   |   70 +
 .../IfExprDecimal64ScalarDecimal64Column.java   |   71 +
 .../IfExprDecimal64ScalarDecimal64Scalar.java   |   75 +
 .../expressions/IfExprLongColumnLongColumn.java |6 +-
 ...fExprStringGroupColumnStringGroupColumn.java |4 +-
 .../IfExprStringGroupColumnStringScalar.java|4 +-
 .../IfExprStringScalarStringGroupColumn.java|4 +-
 .../IfExprStringScalarStringScalar.java |4 +-
 .../IfExprVarCharScalarStringScalar.java|2 +-
 .../expressions/LongColEqualLongColumn.java |  146 -
 .../expressions/LongColEqualLongScalar.java |  157 -
 .../LongColGreaterEqualLongColumn.java  |  146 -
 .../LongColGreaterEqualLongScalar.java  |  158 -
 .../expressions/LongColGreaterLongColumn.java   |  146 -
 .../expressions/LongColGreaterLongScalar.java   |  157 -
 .../expressions/LongColLessEqualLongColumn.java |  146 -
 .../expressions/LongColLessEqualLongScalar.java |  158 -
 .../expressions/LongColLessLongColumn.java  |  146 -
 .../expressions/LongColLessLongScalar.java  |  158 -
 .../expressions/LongColNotEqualLongColumn.java  |  146 -
 .../expressions/LongColNotEqualLongScalar.java  |  158 -
 .../expressions/LongScalarEqualLongColumn.java  |  157 -
 .../LongScalarGreaterEqualLongColumn.java   |  160 -
 .../LongScalarGreaterLongColumn.java|  161 -
 .../LongScalarLessEqualLongColumn.java  |  160 -
 .../expressions/LongScalarLessLongColumn.java   |  161 -
 .../LongScalarNotEqualLongColumn.java   |  161 -
 .../hive/ql/udf/generic/GenericUDFIf.java   |   16 +
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |8 +-
 .../generic/GenericUDFOPEqualOrGreaterThan.java |8 +-
 .../generic/GenericUDFOPEqualOrLessThan.java|8 +-
 .../ql/udf/generic/GenericUDFOPGreaterThan.java |8 +-
 .../ql/udf/generic/GenericUDFOPLessThan.java|8 +-
 .../ql/udf/generic/GenericUDFOPNotEqual.java|8 +-
 .../ql/udf/generic/GenericUDFTimestamp.java |3 +-
 .../exec/vector/TestVectorizationContext.java   |4 +-
 .../expressions/TestVectorArithmetic.java   |   11 +
 .../expressions/TestVectorCastStatement.java|   42 +-
 .../expressions/TestVectorDateAddSub.java   |   10 +
 .../vector/expressions/TestVectorDateDiff.java  |   11 +
 .../expressions/TestVectorFilterCompare.java|  795 
 .../expressions/TestVectorIfStatement.java  |   74 +-
 .../vector/expressions/TestVectorNegative.java  |9 +
 .../expressions/TestVectorStringConcat.java |   11 +
 .../expressions/TestVectorStringUnary.java  |   12 +-
 .../vector/expressions/TestVectorSubStr.java|9 +
 .../expressions/TestVectorTimestampExtract.java |9 +
 .../convert_decimal64_to_decimal.q.out  |6 +-
 .../llap/convert_decimal64_to_decimal.q.out |   10 +-
 .../llap/vector_case_when_1.q.out   |   12 +-
 .../llap/vector_decimal_mapjoin.q.out

[1/2] hive git commit: HIVE-20069: Fix reoptimization in case of DPP and Semijoin optimization (Zoltan Haindrich reviewed by Ashutosh Chauhan)

2018-07-25 Thread kgyrtkirk
Repository: hive
Updated Branches:
  refs/heads/branch-3 7eba98837 -> 9326ba696


HIVE-20069: Fix reoptimization in case of DPP and Semijoin optimization (Zoltan 
Haindrich reviewed by Ashutosh Chauhan)

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

Branch: refs/heads/branch-3
Commit: da3b516d84c28fe76e2e70a5652c9c25839ed365
Parents: 7eba988
Author: Zoltan Haindrich 
Authored: Mon Jul 23 13:54:09 2018 +0200
Committer: Zoltan Haindrich 
Committed: Wed Jul 25 10:54:57 2018 +0200

--
 .../test/resources/testconfiguration.properties |   2 +
 .../hadoop/hive/ql/parse/TezCompiler.java   |  72 
 .../hive/ql/plan/mapper/StatsSources.java   |   9 +-
 .../hadoop/hive/ql/stats/OperatorStats.java |   7 +
 ql/src/test/queries/clientpositive/reopt_dpp.q  |  62 +++
 .../queries/clientpositive/reopt_semijoin.q |  76 
 .../results/clientpositive/llap/reopt_dpp.q.out | 259 
 .../clientpositive/llap/reopt_semijoin.q.out| 420 +++
 8 files changed, 905 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/da3b516d/itests/src/test/resources/testconfiguration.properties
--
diff --git a/itests/src/test/resources/testconfiguration.properties 
b/itests/src/test/resources/testconfiguration.properties
index d28fa67..863bf95 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -476,6 +476,8 @@ minillaplocal.query.files=\
   bucketmapjoin6.q,\
   bucketmapjoin7.q,\
   bucketpruning1.q,\
+  reopt_dpp.q,\
+  reopt_semijoin.q,\
   retry_failure.q,\
   retry_failure_stat_changes.q,\
   retry_failure_oom.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/da3b516d/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
--
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
index 205e726..cffa176 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
@@ -95,6 +95,7 @@ import 
org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger;
 import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer;
 import 
org.apache.hadoop.hive.ql.optimizer.stats.annotation.AnnotateWithStatistics;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.ColStatistics;
 import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
@@ -107,8 +108,10 @@ import org.apache.hadoop.hive.ql.plan.MoveWork;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.Statistics;
 import org.apache.hadoop.hive.ql.plan.TezWork;
+import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
+import org.apache.hadoop.hive.ql.stats.OperatorStats;
 import org.apache.hadoop.hive.ql.stats.StatsUtils;
 import 
org.apache.hadoop.hive.ql.udf.generic.GenericUDAFBloomFilter.GenericUDAFBloomFilterEvaluator;
 import org.slf4j.Logger;
@@ -220,6 +223,10 @@ public class TezCompiler extends TaskCompiler {
 }
 perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, 
"Shared scans optimization");
 
+perfLogger.PerfLogBegin(this.getClass().getName(), 
PerfLogger.TEZ_COMPILER);
+markOperatorsWithUnstableRuntimeStats(procCtx);
+perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, 
"markOperatorsWithUnstableRuntimeStats");
+
 // need a new run of the constant folding because we might have created 
lots
 // of "and true and true" conditions.
 // Rather than run the full constant folding just need to shortcut AND/OR 
expressions
@@ -1036,6 +1043,71 @@ public class TezCompiler extends TaskCompiler {
 ogw.startWalking(topNodes, null);
   }
 
+  private static class MarkTsOfSemijoinsAsIncorrect implements NodeProcessor {
+
+private PlanMapper planMapper;
+
+@Override
+public Object process(Node nd, Stack stack, NodeProcessorCtx 
procCtx, Object... nodeOutputs)
+throws SemanticException {
+  ParseContext pCtx = ((OptimizeTezProcContext) procCtx).parseContext;
+  planMapper = pCtx.getContext().getPlanMapper();
+  if (nd instanceof ReduceSinkOperator) {
+ 

[2/2] hive git commit: HIVE-20231: Backport HIVE-19981 to branch-3 (Daniel Voros via Zoltan Haindrich)

2018-07-25 Thread kgyrtkirk
HIVE-20231: Backport HIVE-19981 to branch-3 (Daniel Voros via Zoltan Haindrich)

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

Branch: refs/heads/branch-3
Commit: 9326ba6964da3ffde80169d276373142e978e8df
Parents: da3b516
Author: Daniel Voros 
Authored: Wed Jul 25 11:35:28 2018 +0200
Committer: Zoltan Haindrich 
Committed: Wed Jul 25 11:35:28 2018 +0200

--
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   6 +
 .../test/resources/testconfiguration.properties |   1 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   8 +
 .../clientpositive/external_table_purge.q   | 165 +
 .../llap/external_table_purge.q.out | 635 +++
 .../hadoop/hive/metastore/HiveMetaStore.java|  33 +-
 .../hive/metastore/utils/MetaStoreUtils.java|   8 +-
 7 files changed, 845 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/9326ba69/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 a7b409e..6c2d0ca 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2852,6 +2852,12 @@ public class HiveConf extends Configuration {
 " on the assumption that data changes by external applications may 
have negative effects" +
 " on these operations."),
 
+HIVE_EXTERNALTABLE_PURGE_DEFAULT("hive.external.table.purge.default", 
false,
+"Set to true to set external.table.purge=true on newly created 
external tables," +
+" which will specify that the table data should be deleted when the 
table is dropped." +
+" Set to false maintain existing behavior that external tables do not 
delete data" +
+" when the table is dropped."),
+
 HIVE_ERROR_ON_EMPTY_PARTITION("hive.error.on.empty.partition", false,
 "Whether to throw an exception if dynamic partition insert generates 
empty results."),
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9326ba69/itests/src/test/resources/testconfiguration.properties
--
diff --git a/itests/src/test/resources/testconfiguration.properties 
b/itests/src/test/resources/testconfiguration.properties
index 863bf95..78a03db 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -407,6 +407,7 @@ minillap.query.files=acid_bucket_pruning.q,\
   reduce_deduplicate_distinct.q, \
   remote_script.q,\
   file_with_header_footer.q,\
+  external_table_purge.q,\
   external_table_with_space_in_location_path.q,\
   import_exported_table.q,\
   orc_llap_counters.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/9326ba69/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 26db245..2bd6c5f 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
@@ -68,6 +68,7 @@ import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -12882,6 +12883,13 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
 }
   }
 }
+
+if (isExt && HiveConf.getBoolVar(conf, 
ConfVars.HIVE_EXTERNALTABLE_PURGE_DEFAULT)) {
+  if (retValue.get(MetaStoreUtils.EXTERNAL_TABLE_PURGE) == null) {
+retValue.put(MetaStoreUtils.EXTERNAL_TABLE_PURGE, "true");
+  }
+}
+
 boolean makeInsertOnly = !isTemporaryTable && HiveConf.getBoolVar(conf, 
ConfVars.HIVE_CREATE_TABLES_AS_INSERT_ONLY);
 boolean makeAcid = !isTemporaryTable &&
 MetastoreConf.getBoolVar(conf, 
MetastoreConf.ConfVars.CREATE_TABLES_AS_ACID) &&

http://git-wip-us.apache.org/repos/asf/hive/blob/9326ba69/ql/src/test/queries/clientpositive/external_table_purge.q
-