You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2018/07/23 11:50:12 UTC

[1/3] hive git commit: HIVE-20182: Backport HIVE-20067 to branch-3 (Daniel Voros via Zoltan Haindrich)

Repository: hive
Updated Branches:
  refs/heads/branch-3 9727ca161 -> b429edbdf


HIVE-20182: Backport HIVE-20067 to branch-3 (Daniel Voros via Zoltan Haindrich)

Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>


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

Branch: refs/heads/branch-3
Commit: b429edbdfef3e6d609f0f3e226240b9be94797bf
Parents: 0c5486f
Author: Daniel Voros <da...@gmail.com>
Authored: Mon Jul 23 13:40:34 2018 +0200
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Mon Jul 23 13:49:43 2018 +0200

----------------------------------------------------------------------
 ql/src/test/queries/clientpositive/mm_all.q                 | 1 +
 .../apache/hadoop/hive/metastore/events/InsertEvent.java    | 9 +++++----
 2 files changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b429edbd/ql/src/test/queries/clientpositive/mm_all.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/mm_all.q b/ql/src/test/queries/clientpositive/mm_all.q
index 61dd3e7..a524c29 100644
--- a/ql/src/test/queries/clientpositive/mm_all.q
+++ b/ql/src/test/queries/clientpositive/mm_all.q
@@ -3,6 +3,7 @@
 
 -- MASK_LINEAGE
 
+set hive.metastore.dml.events=true;
 set hive.mapred.mode=nonstrict;
 set hive.explain.user=false;
 set hive.fetch.task.conversion=none;

http://git-wip-us.apache.org/repos/asf/hive/blob/b429edbd/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
index aa014e9..60ad7db 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/InsertEvent.java
@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.hive.metastore.events;
 
-import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hive.metastore.IHMSHandler;
@@ -33,8 +35,7 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.thrift.TException;
 
-import java.util.ArrayList;
-import java.util.List;
+import com.google.common.collect.Lists;
 
 @InterfaceAudience.Public
 @InterfaceStability.Stable
@@ -65,7 +66,7 @@ public class InsertEvent extends ListenerEvent {
     // TODO MS-SPLIT Switch this back once HiveMetaStoreClient is moved.
     //req.setCapabilities(HiveMetaStoreClient.TEST_VERSION);
     req.setCapabilities(new ClientCapabilities(
-      Lists.newArrayList(ClientCapability.TEST_CAPABILITY)));
+        Lists.newArrayList(ClientCapability.TEST_CAPABILITY, ClientCapability.INSERT_ONLY_TABLES)));
     try {
       this.tableObj = handler.get_table_req(req).getTable();
       if (partVals != null) {


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

Posted by kg...@apache.org.
HIVE-20180: Backport HIVE-19759 to branch-3 (Daniel Voros via Zoltan Haindrich)

Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>


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

Branch: refs/heads/branch-3
Commit: 37771941570ee55d0703bcba13096231ebffcbfa
Parents: 9727ca1
Author: Daniel Voros <da...@gmail.com>
Authored: Mon Jul 23 13:39:20 2018 +0200
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Mon Jul 23 13:49:43 2018 +0200

----------------------------------------------------------------------
 .../apache/hadoop/hive/common/ServerUtils.java  |  8 ++++
 .../apache/hive/spark/client/rpc/TestRpc.java   | 39 ++++++++++++++++----
 2 files changed, 39 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/37771941/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java b/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java
index 7979bbe..d7f4b14 100644
--- a/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/ServerUtils.java
@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.hive.common;
 
+import java.io.IOException;
 import java.net.InetAddress;
+import java.net.ServerSocket;
 import java.net.UnknownHostException;
 
 import org.slf4j.Logger;
@@ -77,4 +79,10 @@ public class ServerUtils {
     }
   }
 
+  public static int findFreePort() throws IOException {
+    ServerSocket socket= new ServerSocket(0);
+    int port = socket.getLocalPort();
+    socket.close();
+    return port;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/37771941/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
----------------------------------------------------------------------
diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
index 5653e4d..013bcff 100644
--- a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
+++ b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java
@@ -46,6 +46,7 @@ import io.netty.channel.nio.NioEventLoopGroup;
 import io.netty.util.concurrent.Future;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hive.common.ServerUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -62,6 +63,7 @@ public class TestRpc {
   private Collection<Closeable> closeables;
   private static final Map<String, String> emptyConfig =
       ImmutableMap.of(HiveConf.ConfVars.SPARK_RPC_CHANNEL_LOG_LEVEL.varname, "DEBUG");
+  private static final int RETRY_ACQUIRE_PORT_COUNT = 10;
 
   @Before
   public void setUp() {
@@ -187,10 +189,21 @@ public class TestRpc {
     assertTrue("Port should be within configured port range:" + server1.getPort(), server1.getPort() >= 49152 && server1.getPort() <= 49333);
     IOUtils.closeQuietly(server1);
 
-    int expectedPort = 65535;
-    config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort));
-    RpcServer server2 = new RpcServer(config);
-    assertTrue("Port should match configured one: " + server2.getPort(), server2.getPort() == expectedPort);
+    int expectedPort = ServerUtils.findFreePort();
+    RpcServer server2 = null;
+    for (int i = 0; i < RETRY_ACQUIRE_PORT_COUNT; i++) {
+      try {
+        config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort));
+        server2 = new RpcServer(config);
+        break;
+      } catch (Exception e) {
+        LOG.debug("Error while connecting to port " + expectedPort + " retrying: " + e.getMessage());
+        expectedPort = ServerUtils.findFreePort();
+      }
+    }
+
+    assertNotNull("Unable to create RpcServer with any attempted port", server2);
+    assertEquals("Port should match configured one: " + server2.getPort(), expectedPort, server2.getPort());
     IOUtils.closeQuietly(server2);
 
     config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, "49552-49222,49223,49224-49333");
@@ -204,10 +217,20 @@ public class TestRpc {
     }
 
     // Retry logic
-    expectedPort = 65535;
-    config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort) + ",21-23");
-    RpcServer server3 = new RpcServer(config);
-    assertTrue("Port should match configured one:" + server3.getPort(), server3.getPort() == expectedPort);
+    expectedPort = ServerUtils.findFreePort();
+    RpcServer server3 = null;
+    for (int i = 0; i < RETRY_ACQUIRE_PORT_COUNT; i++) {
+      try {
+        config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort) + ",21-23");
+        server3 = new RpcServer(config);
+        break;
+      } catch (Exception e) {
+        LOG.debug("Error while connecting to port " + expectedPort + " retrying");
+        expectedPort = ServerUtils.findFreePort();
+      }
+    }
+    assertNotNull("Unable to create RpcServer with any attempted port", server3);
+    assertEquals("Port should match configured one:" + server3.getPort(), expectedPort, server3.getPort());
     IOUtils.closeQuietly(server3);
   }
 


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

Posted by kg...@apache.org.
HIVE-20181: Backport HIVE-20045 to branch-3 (Daniel Voros via Zoltan Haindrich)

Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>


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

Branch: refs/heads/branch-3
Commit: 0c5486fbc6f622237078dcc03f100948932b91b8
Parents: 3777194
Author: Daniel Voros <da...@gmail.com>
Authored: Mon Jul 23 13:39:52 2018 +0200
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Mon Jul 23 13:49:43 2018 +0200

----------------------------------------------------------------------
 common/src/java/org/apache/hadoop/hive/conf/HiveConf.java | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0c5486fb/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 fca1635..a7b409e 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -4339,6 +4339,7 @@ public class HiveConf extends Configuration {
         "Comma separated list of configuration options which are immutable at runtime"),
     HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list",
         METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname
+        + "," + DRUID_METADATA_DB_PASSWORD.varname
         // Adding the S3 credentials from Hadoop config to be hidden
         + ",fs.s3.awsAccessKeyId"
         + ",fs.s3.awsSecretAccessKey"