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 2020/05/17 06:30:16 UTC

[hive] 02/03: HIVE-23396: Many fixes and improvements to stabilize tests (Zoltan Haindrich reviewed by Miklos Gergely)

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

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

commit d1286f2f71f0da8fbede250a9dfc2a0a17c33f3f
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Sun May 17 06:15:16 2020 +0000

    HIVE-23396: Many fixes and improvements to stabilize tests (Zoltan Haindrich reviewed by Miklos Gergely)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../metrics/metrics2/TestCodahaleMetrics.java      |  2 +-
 .../hadoop/hive/metastore/txn/TestTxnHandler.java  |  6 +++---
 .../apache/hadoop/hive/ql/metadata/TestHive.java   | 25 ++++++++--------------
 .../hadoop/hive/ql/metadata/TestHiveRemote.java    | 24 +++++++++++++++------
 .../ql/parse/TestReplicationSemanticAnalyzer.java  | 13 +++++------
 .../cli/session/TestSessionManagerMetrics.java     |  2 +-
 standalone-metastore/metastore-server/pom.xml      |  2 +-
 .../hadoop/hive/metastore/HiveMetaStore.java       |  6 ++++--
 .../hadoop/hive/metastore/MetaStoreTestUtils.java  |  2 +-
 .../hadoop/hive/metastore/TestMarkPartition.java   |  2 +-
 .../hive/metastore/client/MetaStoreClientTest.java | 13 ++++-------
 11 files changed, 47 insertions(+), 50 deletions(-)

diff --git a/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java b/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java
index 9c4e475..85ded7e 100644
--- a/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java
+++ b/common/src/test/org/apache/hadoop/hive/common/metrics/metrics2/TestCodahaleMetrics.java
@@ -55,7 +55,7 @@ public class TestCodahaleMetrics {
   private static final Path tmpDir = Paths.get(System.getProperty("java.io.tmpdir"));
   private static File jsonReportFile;
   private static MetricRegistry metricRegistry;
-  private static final long REPORT_INTERVAL_MS = 100;
+  private static final long REPORT_INTERVAL_MS = 2000;
 
   @BeforeClass
   public static void setUp() throws Exception {
diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
index 868da0c..f65619e 100644
--- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
+++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
@@ -1203,7 +1203,7 @@ public class TestTxnHandler {
       LockRequest req = new LockRequest(components, "me", "localhost");
       LockResponse res = txnHandler.lock(req);
       assertTrue(res.getState() == LockState.ACQUIRED);
-      Thread.sleep(10);
+      Thread.sleep(1000);
       txnHandler.performTimeOuts();
       txnHandler.checkLock(new CheckLockRequest(res.getLockid()));
       fail("Told there was a lock, when it should have timed out.");
@@ -1218,7 +1218,7 @@ public class TestTxnHandler {
     long timeout = txnHandler.setTimeout(1);
     try {
       txnHandler.openTxns(new OpenTxnRequest(503, "me", "localhost"));
-      Thread.sleep(10);
+      Thread.sleep(1000);
       txnHandler.performTimeOuts();
       GetOpenTxnsInfoResponse rsp = txnHandler.getOpenTxnsInfo();
       int numAborted = 0;
@@ -1241,7 +1241,7 @@ public class TestTxnHandler {
       request.setReplPolicy("default.*");
       request.setReplSrcTxnIds(response.getTxn_ids());
       OpenTxnsResponse responseRepl = txnHandler.openTxns(request);
-      Thread.sleep(10);
+      Thread.sleep(1000);
       txnHandler.performTimeOuts();
       GetOpenTxnsInfoResponse rsp = txnHandler.getOpenTxnsInfo();
       int numAborted = 0;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
index 5626dbe..49097a0 100755
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
@@ -38,9 +38,7 @@ import org.apache.hadoop.hive.metastore.PartitionDropOptions;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Database;
 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.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
@@ -67,20 +65,15 @@ import org.apache.logging.log4j.core.config.Configuration;
 import org.apache.logging.log4j.core.config.LoggerConfig;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.junit.Assert;
-import org.slf4j.LoggerFactory;
-
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-
-
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
-import org.junit.Before;
-import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
 import org.junit.Test;
 
 /**
@@ -88,13 +81,13 @@ import org.junit.Test;
  *
  */
 public class TestHive {
-  protected Hive hm;
-  protected HiveConf hiveConf;
+  protected static Hive hm;
+  protected static HiveConf hiveConf;
 
-  @Before
-  public void setUp() throws Exception {
+  @BeforeClass
+  public static void setUp() throws Exception {
 
-    hiveConf = new HiveConf(this.getClass());
+    hiveConf = new HiveConf(TestHive.class);
     hm = setUpImpl(hiveConf);
   }
 
@@ -116,8 +109,8 @@ public class TestHive {
     }
   }
 
-  @After
-  public void tearDown() throws Exception {
+  @AfterClass
+  public static void tearDown() throws Exception {
     try {
 
       // disable trash
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java
index df3b2a5..5a695e9 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java
@@ -23,9 +23,11 @@ import java.net.ServerSocket;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.util.StringUtils;
-
+import org.junit.After;
 import org.junit.Before;
+import org.junit.BeforeClass;
 
 /**
  *
@@ -38,15 +40,17 @@ public class TestHiveRemote extends TestHive {
   /**
    * Start a remote metastore and initialize a Hive object pointing at it.
    */
-  @Before
-  public void setUp() throws Exception {
-    super.setUp();
-    hiveConf = new HiveConf(this.getClass());
-    hiveConf
-    .setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+  @BeforeClass
+  public static void setUp() throws Exception {
+    hiveConf = new HiveConf(TestHiveRemote.class);
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
         "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
     MetaStoreTestUtils.startMetaStoreWithRetry(hiveConf);
+  }
 
+  @Before
+  public void before() throws Exception {
+    SessionState.start(hiveConf);
     try {
       hm = Hive.get(hiveConf);
     } catch (Exception e) {
@@ -58,6 +62,12 @@ public class TestHiveRemote extends TestHive {
     }
   }
 
+  @After
+  public void after() throws IOException {
+    SessionState.get().close();
+    hm.close(false);
+  }
+
   /**
    * Cannot control trash in remote metastore, so skip this test
    */
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestReplicationSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestReplicationSemanticAnalyzer.java
index e1c6eed..e52216c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestReplicationSemanticAnalyzer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestReplicationSemanticAnalyzer.java
@@ -31,15 +31,12 @@ import static org.junit.Assert.assertEquals;
 @RunWith(Enclosed.class)
 public class TestReplicationSemanticAnalyzer {
   private static ParseDriver driver = new ParseDriver();
-  private static HiveConf hiveConf;
+  private static HiveConf hiveConf = buildHiveConf();
 
-  private TestReplicationSemanticAnalyzer() {
-  }
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    hiveConf = new HiveConf();
-    hiveConf.setVar(HIVE_QUOTEDID_SUPPORT, Quotation.NONE.stringValue());
+  public static HiveConf buildHiveConf() {
+    HiveConf conf = new HiveConf();
+    conf.setVar(HIVE_QUOTEDID_SUPPORT, Quotation.NONE.stringValue());
+    return conf;
   }
 
   private static ASTNode parse(String command) throws Exception {
diff --git a/service/src/test/org/apache/hive/service/cli/session/TestSessionManagerMetrics.java b/service/src/test/org/apache/hive/service/cli/session/TestSessionManagerMetrics.java
index 2eb07ee..9658471 100644
--- a/service/src/test/org/apache/hive/service/cli/session/TestSessionManagerMetrics.java
+++ b/service/src/test/org/apache/hive/service/cli/session/TestSessionManagerMetrics.java
@@ -382,7 +382,7 @@ public class TestSessionManagerMetrics {
 
     // We're going to wait for the session to be abandoned.
     String currentValue;
-    int count = 5; // how many times we'll sleep before giving up
+    int count = 10; // how many times we'll sleep before giving up
     String expectedValue = "1";
     do {
       // HIVE_SERVER2_SESSION_CHECK_INTERVAL is set to 3 seconds, so we have to wait for at least
diff --git a/standalone-metastore/metastore-server/pom.xml b/standalone-metastore/metastore-server/pom.xml
index 021c25a..ea5f905 100644
--- a/standalone-metastore/metastore-server/pom.xml
+++ b/standalone-metastore/metastore-server/pom.xml
@@ -682,7 +682,7 @@
             <configuration>
               <sources>
                 <source>src/gen/thrift/gen-javabean</source>
-                <source>${project.build.directory}/generated-sources</source>
+                <source>build/generated-sources</source>
                 <source>src/gen/version</source>
               </sources>
             </configuration>
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 915618c..57c006b 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
@@ -912,8 +912,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       } catch (JDOException e) {
         LOG.warn("Retrying creating default database after error: " + e.getMessage(), e);
         try {
-          createDefaultDB_core(getMS());
-        } catch (InvalidObjectException e1) {
+          RawStore ms = getMS();
+          createDefaultCatalog(ms, wh);
+          createDefaultDB_core(ms);
+        } catch (InvalidObjectException | InvalidOperationException e1) {
           throw new MetaException(e1.getMessage());
         }
       } catch (InvalidObjectException|InvalidOperationException e) {
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
index 2702e69..0e2c35a 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
@@ -176,7 +176,7 @@ public class MetaStoreTestUtils {
         String jdbcUrl = MetastoreConf.getVar(conf, ConfVars.CONNECT_URL_KEY);
         if (!keepJdbcUri) {
           // Setting metastore instance specific jdbc url postfixed with port
-          jdbcUrl = "jdbc:derby:;databaseName=" + TMP_DIR + File.separator
+          jdbcUrl = "jdbc:derby:memory:" + TMP_DIR + File.separator
               + MetaStoreServerUtils.JUNIT_DATABASE_PREFIX + "_" + metaStorePort + ";create=true";
           MetastoreConf.setVar(conf, ConfVars.CONNECT_URL_KEY, jdbcUrl);
         }
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
index 00fae25..811932f 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
@@ -83,7 +83,7 @@ public class TestMarkPartition {
     kvs.put("b", "'2011'");
     msc.markPartitionForEvent(dbName, tableName, kvs, PartitionEventType.LOAD_DONE);
     Assert.assertTrue(msc.isPartitionMarkedForEvent(dbName, tableName, kvs, PartitionEventType.LOAD_DONE));
-    Thread.sleep(3000);
+    Thread.sleep(10000);
     Assert.assertFalse(msc.isPartitionMarkedForEvent(dbName, tableName, kvs, PartitionEventType.LOAD_DONE));
 
     kvs.put("b", "'2012'");
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/MetaStoreClientTest.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/MetaStoreClientTest.java
index afe60b5..67fb749 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/MetaStoreClientTest.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/client/MetaStoreClientTest.java
@@ -33,7 +33,6 @@ import org.slf4j.LoggerFactory;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
@@ -47,7 +46,7 @@ public abstract class MetaStoreClientTest {
   // Needed until there is no junit release with @BeforeParam, @AfterParam (junit 4.13)
   // https://github.com/junit-team/junit4/commit/1bf8438b65858565dbb64736bfe13aae9cfc1b5a
   // Then we should remove our own copy
-  private static Set<AbstractMetaStoreService> metaStoreServices = null;
+  private static List<AbstractMetaStoreService> metaStoreServices = null;
 
   @Rule
   public TestRule ignoreRule;
@@ -57,7 +56,7 @@ public abstract class MetaStoreClientTest {
     List<Object[]> result = MetaStoreFactoryForTests.getMetaStores();
     metaStoreServices = result.stream()
         .map(test -> (AbstractMetaStoreService)test[1])
-        .collect(Collectors.toSet());
+        .collect(Collectors.toList());
     return result;
   }
 
@@ -78,9 +77,7 @@ public abstract class MetaStoreClientTest {
       try {
         metaStoreService.start(msConf, extraConf);
       } catch(Exception e) {
-        // Catch the exceptions, so every other metastore could be stopped as well
-        // Log it, so at least there is a slight possibility we find out about this :)
-        LOG.error("Error starting MetaStoreService", e);
+        throw new RuntimeException("Error starting MetaStoreService", e);
       }
     }
   }
@@ -91,9 +88,7 @@ public abstract class MetaStoreClientTest {
       try {
         metaStoreService.stop();
       } catch(Exception e) {
-        // Catch the exceptions, so every other metastore could be stopped as well
-        // Log it, so at least there is a slight possibility we find out about this :)
-        LOG.error("Error stopping MetaStoreService", e);
+        throw new RuntimeException("Error stopping MetaStoreService", e);
       }
     }
   }