hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ych...@apache.org
Subject hive git commit: HIVE-16242: Run BeeLine tests parallel (Peter Vary via Yongzhi Chen)
Date Fri, 24 Mar 2017 13:45:19 GMT
Repository: hive
Updated Branches:
  refs/heads/master 440c9c723 -> 5eb02b939


HIVE-16242: Run BeeLine tests parallel (Peter Vary via Yongzhi Chen)


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

Branch: refs/heads/master
Commit: 5eb02b9392eeb8309d0e071a22d0319507bb63b3
Parents: 440c9c7
Author: Yongzhi Chen <ychena@apache.org>
Authored: Fri Mar 24 09:42:13 2017 -0400
Committer: Yongzhi Chen <ychena@apache.org>
Committed: Fri Mar 24 09:42:13 2017 -0400

----------------------------------------------------------------------
 .../hadoop/hive/cli/TestBeeLineDriver.java      |  4 +-
 .../org/apache/hive/beeline/Parallelized.java   | 64 ++++++++++++++++++++
 .../org/apache/hive/beeline/qfile/QFile.java    |  2 +
 3 files changed, 68 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5eb02b93/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java
----------------------------------------------------------------------
diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java
b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java
index 24eeb9d..4dd17c1e 100644
--- a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java
+++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestBeeLineDriver.java
@@ -22,15 +22,15 @@ import java.util.List;
 
 import org.apache.hadoop.hive.cli.control.CliAdapter;
 import org.apache.hadoop.hive.cli.control.CliConfigs;
+import org.apache.hive.beeline.Parallelized;
 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 org.junit.runners.Parameterized.Parameters;
 
-@RunWith(Parameterized.class)
+@RunWith(Parallelized.class)
 public class TestBeeLineDriver {
 
   static CliAdapter adapter = new CliConfigs.BeeLineConfig().getCliAdapter();

http://git-wip-us.apache.org/repos/asf/hive/blob/5eb02b93/itests/util/src/main/java/org/apache/hive/beeline/Parallelized.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hive/beeline/Parallelized.java b/itests/util/src/main/java/org/apache/hive/beeline/Parallelized.java
new file mode 100644
index 0000000..a819e26
--- /dev/null
+++ b/itests/util/src/main/java/org/apache/hive/beeline/Parallelized.java
@@ -0,0 +1,64 @@
+/**
+ * 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.hive.beeline;
+
+import org.junit.runners.Parameterized;
+import org.junit.runners.model.RunnerScheduler;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Class to run Parameterized test in parallel.
+ * Source: http://hwellmann.blogspot.hu/2009/12/running-parameterized-junit-tests-in.html
+ */
+public class Parallelized extends Parameterized {
+  private static class ThreadPoolScheduler implements RunnerScheduler {
+    private ExecutorService executor;
+
+    public ThreadPoolScheduler() {
+      String threads = System.getProperty("junit.parallel.threads");
+      int numThreads = Runtime.getRuntime().availableProcessors();
+      if (threads != null) {
+        numThreads = Integer.parseInt(threads);
+      }
+      executor = Executors.newFixedThreadPool(numThreads);
+    }
+
+    @Override
+    public void finished() {
+      executor.shutdown();
+      try {
+        executor.awaitTermination(10, TimeUnit.MINUTES);
+      } catch (InterruptedException exc) {
+        throw new RuntimeException(exc);
+      }
+    }
+
+    @Override
+    public void schedule(Runnable childStatement) {
+      executor.submit(childStatement);
+    }
+  }
+
+  public Parallelized(Class klass) throws Throwable {
+    super(klass);
+    setScheduler(new ThreadPoolScheduler());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/5eb02b93/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java b/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java
index 49d6d24..91de99b 100644
--- a/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java
+++ b/itests/util/src/main/java/org/apache/hive/beeline/qfile/QFile.java
@@ -188,6 +188,8 @@ public final class QFile {
 
     return new RegexFilterSet()
         .addFilter(logPattern, "")
+        .addFilter("(?s)\nWaiting to acquire compile lock:.*?Acquired the compile lock.\n",
+            "\nAcquired the compile lock.\n")
         .addFilter("Getting log thread is interrupted, since query is done!\n", "")
         .addFilter("going to print operations logs\n", "")
         .addFilter("printed operations logs\n", "")


Mime
View raw message