You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ai...@apache.org on 2016/12/12 14:24:04 UTC

hive git commit: HIVE-14960: Improve the stability of TestNotificationListener (Marta Kuczora via Aihua Xu)

Repository: hive
Updated Branches:
  refs/heads/master 6ef0b5078 -> d556689e3


HIVE-14960: Improve the stability of TestNotificationListener (Marta Kuczora via Aihua Xu)


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

Branch: refs/heads/master
Commit: d556689e3b3379e1abcef589a54d2d12c8b63c5e
Parents: 6ef0b50
Author: Aihua Xu <ai...@apache.org>
Authored: Mon Dec 12 09:23:24 2016 -0500
Committer: Aihua Xu <ai...@apache.org>
Committed: Mon Dec 12 09:23:24 2016 -0500

----------------------------------------------------------------------
 .../listener/TestNotificationListener.java      | 29 ++++++++++++++------
 1 file changed, 20 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d556689e/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java b/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
index 9e03da4..ef7b575 100644
--- a/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
+++ b/hcatalog/server-extensions/src/test/java/org/apache/hive/hcatalog/listener/TestNotificationListener.java
@@ -24,6 +24,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Vector;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import javax.jms.Connection;
 import javax.jms.ConnectionFactory;
@@ -65,6 +67,18 @@ import org.junit.Test;
 public class TestNotificationListener extends HCatBaseTest implements MessageListener {
 
   private List<String> actualMessages = new Vector<String>();
+  private static final int MSG_RECEIVED_TIMEOUT = 30;
+  private static final List<String> expectedMessages = Arrays.asList(
+      HCatConstants.HCAT_CREATE_DATABASE_EVENT,
+      HCatConstants.HCAT_CREATE_TABLE_EVENT,
+      HCatConstants.HCAT_ADD_PARTITION_EVENT,
+      HCatConstants.HCAT_ALTER_PARTITION_EVENT,
+      HCatConstants.HCAT_DROP_PARTITION_EVENT,
+      HCatConstants.HCAT_ALTER_TABLE_EVENT,
+      HCatConstants.HCAT_DROP_TABLE_EVENT,
+      HCatConstants.HCAT_DROP_DATABASE_EVENT);
+  private static final CountDownLatch messageReceivedSignal =
+      new CountDownLatch(expectedMessages.size());
 
   @Before
   public void setUp() throws Exception {
@@ -105,15 +119,6 @@ public class TestNotificationListener extends HCatBaseTest implements MessageLis
 
   @After
   public void tearDown() throws Exception {
-    List<String> expectedMessages = Arrays.asList(
-        HCatConstants.HCAT_CREATE_DATABASE_EVENT,
-        HCatConstants.HCAT_CREATE_TABLE_EVENT,
-        HCatConstants.HCAT_ADD_PARTITION_EVENT,
-        HCatConstants.HCAT_ALTER_PARTITION_EVENT,
-        HCatConstants.HCAT_DROP_PARTITION_EVENT,
-        HCatConstants.HCAT_ALTER_TABLE_EVENT,
-        HCatConstants.HCAT_DROP_TABLE_EVENT,
-        HCatConstants.HCAT_DROP_DATABASE_EVENT);
     Assert.assertEquals(expectedMessages, actualMessages);
   }
 
@@ -132,6 +137,9 @@ public class TestNotificationListener extends HCatBaseTest implements MessageLis
     driver.run("alter table mytbl add columns (c int comment 'this is an int', d decimal(3,2))");
     driver.run("drop table mytbl");
     driver.run("drop database mydb");
+
+    // Wait until either all messages are processed or a maximum time limit is reached.
+    messageReceivedSignal.await(MSG_RECEIVED_TIMEOUT, TimeUnit.SECONDS);
   }
 
   @Override
@@ -248,5 +256,8 @@ public class TestNotificationListener extends HCatBaseTest implements MessageLis
       e.printStackTrace(System.err);
       assert false;
     }
+    finally {
+      messageReceivedSignal.countDown();
+    }
   }
 }