You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by dm...@apache.org on 2020/07/22 13:40:21 UTC

[hive] branch master updated: HIVE-23797: Throw exception when no metastore found in zookeeper (Zhihua Deng, reviewed by David Mollitor)

This is an automated email from the ASF dual-hosted git repository.

dmollitor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 9b50557  HIVE-23797: Throw exception when no metastore found in zookeeper (Zhihua Deng, reviewed by David Mollitor)
9b50557 is described below

commit 9b505575ca420340c41ccb5acabd85a8803cbd45
Author: dengzh <de...@gmail.com>
AuthorDate: Wed Jul 22 21:40:12 2020 +0800

    HIVE-23797: Throw exception when no metastore found in zookeeper (Zhihua Deng, reviewed by David Mollitor)
---
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |  7 ++
 .../hive/metastore/TestRemoteHMSZKNegative.java    | 81 ++++++++++++++++++++++
 2 files changed, 88 insertions(+)

diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index a8d37c5..84a86c5 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -327,6 +327,13 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
 
+    if (metastoreUrisString.isEmpty() && "zookeeper".equalsIgnoreCase(serviceDiscoveryMode)) {
+      throw new MetaException("No metastore service discovered in ZooKeeper. "
+          + "Please ensure that at least one metastore server is online");
+    }
+
+    LOG.info("Resolved metastore uris: {}", metastoreUrisString);
+
     List<URI> metastoreURIArray = new ArrayList<URI>();
     try {
       for (String s : metastoreUrisString) {
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHMSZKNegative.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHMSZKNegative.java
new file mode 100644
index 0000000..e860691
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHMSZKNegative.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.ZooKeeperHiveHelper;
+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.CreateMode;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test MetaStore Client throws exception when no MetaStore spec found in zookeeper.
+ *
+ */
+public class TestRemoteHMSZKNegative {
+  private TestingServer zkServer;
+  private Configuration conf;
+  private CuratorFramework zkClient;
+  private String rootNamespace = this.getClass().getSimpleName();
+
+  @Before
+  public void setUp() throws Exception {
+    zkServer = new TestingServer();
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, zkServer.getConnectString());
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_ZOOKEEPER_NAMESPACE, rootNamespace);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_SERVICE_DISCOVERY_MODE, "zookeeper");
+    zkClient = CuratorFrameworkFactory.newClient(zkServer.getConnectString(), new RetryOneTime(2000));
+    zkClient.start();
+    zkClient.create()
+        .creatingParentsIfNeeded()
+        .withMode(CreateMode.PERSISTENT)
+        .forPath(ZooKeeperHiveHelper.ZOOKEEPER_PATH_SEPARATOR + rootNamespace);
+  }
+
+  @Test
+  public void createClient() {
+    try {
+      new HiveMetaStoreClient(conf);
+      Assert.fail("an exception is expected");
+    } catch (Exception e) {
+      Assert.assertTrue(e instanceof MetaException);
+      Assert.assertTrue(e.getMessage().contains("No metastore service discovered in ZooKeeper"));
+    }
+  }
+
+  @After
+  public void stop() throws Exception {
+    if (zkClient != null) {
+      zkClient.close();
+    }
+    if (zkServer != null) {
+      zkServer.stop();
+    }
+  }
+}