From abb07a126bf59cb3954f6b01b29f38d636d56e1f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 15 Sep 2015 13:05:23 -0700 Subject: [PATCH 1/3] [SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py. --- core/pom.xml | 10 ------- dev/run-tests.py | 19 ++++++++++++-- dev/sparktestsupport/modules.py | 24 ++++++++++++++++- external/flume/pom.xml | 10 ------- external/kafka/pom.xml | 10 ------- external/mqtt/pom.xml | 10 ------- external/twitter/pom.xml | 10 ------- external/zeromq/pom.xml | 10 ------- extras/java8-tests/pom.xml | 10 ------- extras/kinesis-asl/pom.xml | 5 ---- launcher/pom.xml | 5 ---- mllib/pom.xml | 10 ------- network/common/pom.xml | 10 ------- network/shuffle/pom.xml | 10 ------- pom.xml | 17 ++++++++++-- project/SparkBuild.scala | 13 ++++++++-- sql/core/pom.xml | 5 ---- .../execution/HiveCompatibilitySuite.scala | 2 ++ sql/hive/pom.xml | 5 ---- .../spark/sql/hive/ExtendedHiveTest.java | 26 +++++++++++++++++++ .../spark/sql/hive/client/VersionsSuite.scala | 2 ++ streaming/pom.xml | 10 ------- unsafe/pom.xml | 10 ------- .../spark/deploy/yarn/ExtendedYarnTest.java | 26 +++++++++++++++++++ .../spark/deploy/yarn/YarnClusterSuite.scala | 1 + .../yarn/YarnShuffleIntegrationSuite.scala | 1 + 26 files changed, 124 insertions(+), 147 deletions(-) create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/ExtendedHiveTest.java create mode 100644 yarn/src/test/java/org/apache/spark/deploy/yarn/ExtendedYarnTest.java diff --git a/core/pom.xml b/core/pom.xml index e31d90f60889..8a2018109622 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -331,16 +331,6 @@ scalacheck_${scala.binary.version} test - - junit - junit - test - - - com.novocode - junit-interface - test - org.apache.curator curator-test diff --git a/dev/run-tests.py b/dev/run-tests.py index d8b22e1665e7..1a816585187d 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -118,6 +118,14 @@ def determine_modules_to_test(changed_modules): return modules_to_test.union(set(changed_modules)) +def determine_tags_to_exclude(changed_modules): + tags = [] + for m in modules.all_modules: + if m not in changed_modules: + tags += m.test_tags + return tags + + # ------------------------------------------------------------------------------------------------- # Functions for working with subprocesses and shell tools # ------------------------------------------------------------------------------------------------- @@ -369,6 +377,7 @@ def detect_binary_inop_with_mima(): def run_scala_tests_maven(test_profiles): mvn_test_goals = ["test", "--fail-at-end"] + profiles_and_goals = test_profiles + mvn_test_goals print("[info] Running Spark tests using Maven with these arguments: ", @@ -392,7 +401,7 @@ def run_scala_tests_sbt(test_modules, test_profiles): exec_sbt(profiles_and_goals) -def run_scala_tests(build_tool, hadoop_version, test_modules): +def run_scala_tests(build_tool, hadoop_version, test_modules, excluded_tags): """Function to properly execute all tests passed in as a set from the `determine_test_suites` function""" set_title_and_block("Running Spark unit tests", "BLOCK_SPARK_UNIT_TESTS") @@ -401,6 +410,10 @@ def run_scala_tests(build_tool, hadoop_version, test_modules): test_profiles = get_hadoop_profiles(hadoop_version) + \ list(set(itertools.chain.from_iterable(m.build_profile_flags for m in test_modules))) + + if excluded_tags: + test_profiles += ['-Dtest.exclude.tags=' + ",".join(excluded_tags)] + if build_tool == "maven": run_scala_tests_maven(test_profiles) else: @@ -500,8 +513,10 @@ def main(): target_branch = os.environ["ghprbTargetBranch"] changed_files = identify_changed_files_from_git_commits("HEAD", target_branch=target_branch) changed_modules = determine_modules_for_files(changed_files) + excluded_tags = determine_tags_to_exclude(changed_modules) if not changed_modules: changed_modules = [modules.root] + excluded_tags = [] print("[info] Found the following changed modules:", ", ".join(x.name for x in changed_modules)) @@ -541,7 +556,7 @@ def main(): detect_binary_inop_with_mima() # run the test suites - run_scala_tests(build_tool, hadoop_version, test_modules) + run_scala_tests(build_tool, hadoop_version, test_modules, excluded_tags) modules_with_python_tests = [m for m in test_modules if m.python_test_goals] if modules_with_python_tests: diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 346452f3174e..65397f1f3e0b 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -31,7 +31,7 @@ class Module(object): def __init__(self, name, dependencies, source_file_regexes, build_profile_flags=(), environ={}, sbt_test_goals=(), python_test_goals=(), blacklisted_python_implementations=(), - should_run_r_tests=False): + test_tags=(), should_run_r_tests=False): """ Define a new module. @@ -50,6 +50,8 @@ def __init__(self, name, dependencies, source_file_regexes, build_profile_flags= :param blacklisted_python_implementations: A set of Python implementations that are not supported by this module's Python components. The values in this set should match strings returned by Python's `platform.python_implementation()`. + :param test_tags A set of tags that will be excluded when running unit tests if the module + is not explicitly changed. :param should_run_r_tests: If true, changes in this module will trigger all R tests. """ self.name = name @@ -60,6 +62,7 @@ def __init__(self, name, dependencies, source_file_regexes, build_profile_flags= self.environ = environ self.python_test_goals = python_test_goals self.blacklisted_python_implementations = blacklisted_python_implementations + self.test_tags = test_tags self.should_run_r_tests = should_run_r_tests self.dependent_modules = set() @@ -85,6 +88,9 @@ def contains_file(self, filename): "catalyst/test", "sql/test", "hive/test", + ], + test_tags=[ + "org.apache.spark.sql.hive.ExtendedHiveTest" ] ) @@ -398,6 +404,22 @@ def contains_file(self, filename): ) +yarn = Module( + name="yarn", + dependencies=[], + source_file_regexes=[ + "yarn/", + "network/yarn/", + ], + sbt_test_goals=[ + "yarn/test", + "network-yarn/test", + ], + test_tags=[ + "org.apache.spark.deploy.yarn.ExtendedYarnTest" + ] +) + # The root module is a dummy module which is used to run all of the tests. # No other modules should directly depend on this module. root = Module( diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 132062f94fb4..3154e36c21ef 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -66,16 +66,6 @@ scalacheck_${scala.binary.version} test - - junit - junit - test - - - com.novocode - junit-interface - test - target/scala-${scala.binary.version}/classes diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 05abd9e2e681..7d0d46dadc72 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -86,16 +86,6 @@ scalacheck_${scala.binary.version} test - - junit - junit - test - - - com.novocode - junit-interface - test - target/scala-${scala.binary.version}/classes diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 05e6338a08b0..913c47d33f48 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -58,16 +58,6 @@ scalacheck_${scala.binary.version} test - - junit - junit - test - - - com.novocode - junit-interface - test - org.apache.activemq activemq-core diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 244ad58ae959..9137bf25ee8a 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -58,16 +58,6 @@ scalacheck_${scala.binary.version} test - - junit - junit - test - - - com.novocode - junit-interface - test - target/scala-${scala.binary.version}/classes diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 171df8682c84..6fec4f0e8a0f 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -57,16 +57,6 @@ scalacheck_${scala.binary.version} test - - junit - junit - test - - - com.novocode - junit-interface - test - target/scala-${scala.binary.version}/classes diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 81794a853631..dba3dda8a956 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -58,16 +58,6 @@ test-jar test - - junit - junit - test - - - com.novocode - junit-interface - test - diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 6dd8ff69c294..760f183a2ef3 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -74,11 +74,6 @@ scalacheck_${scala.binary.version} test - - com.novocode - junit-interface - test - target/scala-${scala.binary.version}/classes diff --git a/launcher/pom.xml b/launcher/pom.xml index ed38e66aa246..80696280a1d1 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -42,11 +42,6 @@ log4j test - - junit - junit - test - org.mockito mockito-core diff --git a/mllib/pom.xml b/mllib/pom.xml index 22c0c6008ba3..5dedacb38874 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -94,16 +94,6 @@ scalacheck_${scala.binary.version} test - - junit - junit - test - - - com.novocode - junit-interface - test - org.mockito mockito-core diff --git a/network/common/pom.xml b/network/common/pom.xml index 1cc054a8936c..9c12cca0df60 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -64,16 +64,6 @@ - - junit - junit - test - - - com.novocode - junit-interface - test - log4j log4j diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index 7a66c968041c..e4f4c57b683c 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -78,16 +78,6 @@ test-jar test - - junit - junit - test - - - com.novocode - junit-interface - test - log4j log4j diff --git a/pom.xml b/pom.xml index 653599464114..2927d3e10756 100644 --- a/pom.xml +++ b/pom.xml @@ -181,6 +181,7 @@ 0.9.2 ${java.home} + @@ -1952,6 +1964,7 @@ __not_used__ + ${test.exclude.tags} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 901cfa538d23..d80d300f1c3b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -567,11 +567,20 @@ object TestSettings { javaOptions in Test ++= "-Xmx3g -Xss4096k -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g" .split(" ").toSeq, javaOptions += "-Xmx3g", + // Exclude tags defined in a system property + testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, + sys.props.get("test.exclude.tags").map { tags => + tags.split(",").flatMap { tag => Seq("-l", tag) }.toSeq + }.getOrElse(Nil): _*), + testOptions in Test += Tests.Argument(TestFrameworks.JUnit, + sys.props.get("test.exclude.tags").map { tags => + Seq("--exclude-categories=" + tags) + }.getOrElse(Nil): _*), // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), - testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), + testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), // Enable Junit testing. - libraryDependencies += "com.novocode" % "junit-interface" % "0.9" % "test", + libraryDependencies += "com.novocode" % "junit-interface" % "0.11" % "test", // Only allow one test at a time, even across projects, since they run in the same JVM parallelExecution in Test := false, // Make sure the test temp directory exists. diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 465aa3a3888c..fa6732db183d 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -73,11 +73,6 @@ jackson-databind ${fasterxml.jackson.version} - - junit - junit - test - org.scalacheck scalacheck_${scala.binary.version} diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index ab309e0a1d36..ffc4c32794ca 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -24,11 +24,13 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.scalatest.BeforeAndAfter import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.hive.ExtendedHiveTest import org.apache.spark.sql.hive.test.TestHive /** * Runs the test cases that are included in the hive distribution. */ +@ExtendedHiveTest class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // TODO: bundle in jar files... get from classpath private lazy val hiveQueryDir = TestHive.getHiveFile( diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ac67fe5f47be..82cfeb2bb95d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -160,11 +160,6 @@ scalacheck_${scala.binary.version} test - - junit - junit - test - org.apache.spark spark-sql_${scala.binary.version} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/ExtendedHiveTest.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/ExtendedHiveTest.java new file mode 100644 index 000000000000..e2183183fb55 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/ExtendedHiveTest.java @@ -0,0 +1,26 @@ +/* + * 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.spark.sql.hive; + +import java.lang.annotation.*; +import org.scalatest.TagAnnotation; + +@TagAnnotation +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface ExtendedHiveTest { } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index f0bb77092c0c..888d1b7b4553 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.hive.HiveContext import org.apache.spark.{Logging, SparkFunSuite} import org.apache.spark.sql.catalyst.expressions.{NamedExpression, Literal, AttributeReference, EqualTo} import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.hive.ExtendedHiveTest import org.apache.spark.sql.types.IntegerType import org.apache.spark.util.Utils @@ -32,6 +33,7 @@ import org.apache.spark.util.Utils * sure that reflective calls are not throwing NoSuchMethod error, but the actually functionality * is not fully tested. */ +@ExtendedHiveTest class VersionsSuite extends SparkFunSuite with Logging { // Do not use a temp path here to speed up subsequent executions of the unit test during diff --git a/streaming/pom.xml b/streaming/pom.xml index 5cc9001b0e9a..1e6ee009ca6d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -84,21 +84,11 @@ scalacheck_${scala.binary.version} test - - junit - junit - test - org.seleniumhq.selenium selenium-java test - - com.novocode - junit-interface - test - target/scala-${scala.binary.version}/classes diff --git a/unsafe/pom.xml b/unsafe/pom.xml index 066abe92e51c..4e8b9a84bb67 100644 --- a/unsafe/pom.xml +++ b/unsafe/pom.xml @@ -55,16 +55,6 @@ - - junit - junit - test - - - com.novocode - junit-interface - test - org.mockito mockito-core diff --git a/yarn/src/test/java/org/apache/spark/deploy/yarn/ExtendedYarnTest.java b/yarn/src/test/java/org/apache/spark/deploy/yarn/ExtendedYarnTest.java new file mode 100644 index 000000000000..7a8f2fe979c1 --- /dev/null +++ b/yarn/src/test/java/org/apache/spark/deploy/yarn/ExtendedYarnTest.java @@ -0,0 +1,26 @@ +/* + * 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.spark.deploy.yarn; + +import java.lang.annotation.*; +import org.scalatest.TagAnnotation; + +@TagAnnotation +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface ExtendedYarnTest { } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index b5a42fd6afd9..105c3090d489 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -39,6 +39,7 @@ import org.apache.spark.util.Utils * applications, and require the Spark assembly to be built before they can be successfully * run. */ +@ExtendedYarnTest class YarnClusterSuite extends BaseYarnClusterSuite { override def newYarnConfig(): YarnConfiguration = new YarnConfiguration() diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 8d9c9b3004ed..4700e2428df0 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} /** * Integration test for the external shuffle service with a yarn mini-cluster */ +@ExtendedYarnTest class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { override def newYarnConfig(): YarnConfiguration = { From 9e0b3c4fee3b88b012efc479d4831034e144e04b Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 15 Sep 2015 15:21:33 -0700 Subject: [PATCH 2/3] Move test tags to a separate module. A new module ("tags/") was added where all test tags should be placed. This is required because the JUnit runners for both maven and sbt require the classes to be visible when you disable them, even if no tests actually reference them. To avoid a cyclic dependency, the tags dependency needs to be manually added to all poms, which is annoying. An alternative would be to make the tag module's pom not have spark-parent as its parent, but that would mean replicating some information from the parent pom and fixing other code (such as the "change-scala-version.sh" script). --- bagel/pom.xml | 4 ++ core/pom.xml | 4 ++ dev/sparktestsupport/modules.py | 4 +- external/flume-sink/pom.xml | 4 ++ external/flume/pom.xml | 4 ++ external/kafka/pom.xml | 4 ++ external/mqtt/pom.xml | 4 ++ external/twitter/pom.xml | 4 ++ external/zeromq/pom.xml | 4 ++ extras/java8-tests/pom.xml | 4 ++ extras/kinesis-asl/pom.xml | 4 ++ graphx/pom.xml | 4 ++ launcher/pom.xml | 5 ++ mllib/pom.xml | 4 ++ network/common/pom.xml | 4 ++ network/shuffle/pom.xml | 4 ++ network/yarn/pom.xml | 4 ++ pom.xml | 7 +++ project/SparkBuild.scala | 4 +- repl/pom.xml | 4 ++ sql/catalyst/pom.xml | 4 ++ sql/core/pom.xml | 4 ++ sql/hive-thriftserver/pom.xml | 4 ++ .../execution/HiveCompatibilitySuite.scala | 2 +- sql/hive/pom.xml | 4 ++ .../spark/sql/hive/client/VersionsSuite.scala | 2 +- streaming/pom.xml | 4 ++ tags/pom.xml | 50 +++++++++++++++++++ .../apache/spark/tags}/ExtendedHiveTest.java | 2 +- .../apache/spark/tags}/ExtendedYarnTest.java | 2 +- unsafe/pom.xml | 4 ++ yarn/pom.xml | 4 ++ .../spark/deploy/yarn/YarnClusterSuite.scala | 1 + .../yarn/YarnShuffleIntegrationSuite.scala | 1 + 34 files changed, 164 insertions(+), 8 deletions(-) create mode 100644 tags/pom.xml rename {sql/hive/src/test/java/org/apache/spark/sql/hive => tags/src/main/java/org/apache/spark/tags}/ExtendedHiveTest.java (96%) rename {yarn/src/test/java/org/apache/spark/deploy/yarn => tags/src/main/java/org/apache/spark/tags}/ExtendedYarnTest.java (96%) diff --git a/bagel/pom.xml b/bagel/pom.xml index 3baf8d47b4dc..672e9469aec9 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -52,6 +52,10 @@ scalacheck_${scala.binary.version} test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/core/pom.xml b/core/pom.xml index 8a2018109622..c0af98a04fb1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -352,6 +352,10 @@ py4j 0.8.2.1 + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 65397f1f3e0b..d65547e04db4 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -90,7 +90,7 @@ def contains_file(self, filename): "hive/test", ], test_tags=[ - "org.apache.spark.sql.hive.ExtendedHiveTest" + "org.apache.spark.tags.ExtendedHiveTest" ] ) @@ -416,7 +416,7 @@ def contains_file(self, filename): "network-yarn/test", ], test_tags=[ - "org.apache.spark.deploy.yarn.ExtendedYarnTest" + "org.apache.spark.tags.ExtendedYarnTest" ] ) diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index d7c2ac474a18..75113ff753e7 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -90,6 +90,10 @@ 3.4.0.Final test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 3154e36c21ef..57f83607365d 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -66,6 +66,10 @@ scalacheck_${scala.binary.version} test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 7d0d46dadc72..79258c126e04 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -86,6 +86,10 @@ scalacheck_${scala.binary.version} test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 913c47d33f48..59fba8b826b4 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -64,6 +64,10 @@ 5.7.0 test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 9137bf25ee8a..4c22ec8b3b15 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -58,6 +58,10 @@ scalacheck_${scala.binary.version} test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 6fec4f0e8a0f..02d6b8128157 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -57,6 +57,10 @@ scalacheck_${scala.binary.version} test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index dba3dda8a956..4ce90e75fd35 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -58,6 +58,10 @@ test-jar test + + org.apache.spark + spark-test-tags_${scala.binary.version} + diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 760f183a2ef3..ef72d97eae69 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -74,6 +74,10 @@ scalacheck_${scala.binary.version} test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/graphx/pom.xml b/graphx/pom.xml index 202fc19002d1..987b831021a5 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -66,6 +66,10 @@ scalacheck_${scala.binary.version} test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/launcher/pom.xml b/launcher/pom.xml index 80696280a1d1..d595d74642ab 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -58,6 +58,11 @@ test + + org.apache.spark + spark-test-tags_${scala.binary.version} + + org.apache.hadoop diff --git a/mllib/pom.xml b/mllib/pom.xml index 5dedacb38874..70139121d8c7 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -121,6 +121,10 @@ + + org.apache.spark + spark-test-tags_${scala.binary.version} + diff --git a/network/common/pom.xml b/network/common/pom.xml index 9c12cca0df60..9af6cc5e925f 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -69,6 +69,10 @@ log4j test + + org.apache.spark + spark-test-tags_${scala.binary.version} + org.mockito mockito-core diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index e4f4c57b683c..70ba5cb1995b 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -78,6 +78,10 @@ test-jar test + + org.apache.spark + spark-test-tags_${scala.binary.version} + log4j log4j diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml index e745180eace7..541ed9a8d0ab 100644 --- a/network/yarn/pom.xml +++ b/network/yarn/pom.xml @@ -44,6 +44,10 @@ spark-network-shuffle_${scala.binary.version} ${project.version} + + org.apache.spark + spark-test-tags_${scala.binary.version} + diff --git a/pom.xml b/pom.xml index 2927d3e10756..530bc5dbbb5c 100644 --- a/pom.xml +++ b/pom.xml @@ -86,6 +86,7 @@ + tags core bagel graphx @@ -353,6 +354,12 @@ + + org.apache.spark + spark-test-tags_${scala.binary.version} + ${project.version} + test + com.twitter chill_${scala.binary.version} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d80d300f1c3b..c97ef3b34ef8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -35,11 +35,11 @@ object BuildCommons { val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, sql, networkCommon, networkShuffle, streaming, streamingFlumeSink, streamingFlume, streamingKafka, - streamingMqtt, streamingTwitter, streamingZeromq, launcher, unsafe) = + streamingMqtt, streamingTwitter, streamingZeromq, launcher, unsafe, testTags) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", "sql", "network-common", "network-shuffle", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", - "streaming-zeromq", "launcher", "unsafe").map(ProjectRef(buildLocation, _)) + "streaming-zeromq", "launcher", "unsafe", "test-tags").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, java8Tests, sparkGangliaLgpl, streamingKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl", diff --git a/repl/pom.xml b/repl/pom.xml index 5cf416a4a544..fb0a0e1286c8 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -91,6 +91,10 @@ mockito-core test + + org.apache.spark + spark-test-tags_${scala.binary.version} + diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6cfd53e868f8..61d6fc63554b 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -53,6 +53,10 @@ test-jar test + + org.apache.spark + spark-test-tags_${scala.binary.version} + org.apache.spark spark-unsafe_${scala.binary.version} diff --git a/sql/core/pom.xml b/sql/core/pom.xml index fa6732db183d..c96855e261ee 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -60,6 +60,10 @@ test-jar test + + org.apache.spark + spark-test-tags_${scala.binary.version} + org.apache.parquet parquet-column diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index f7fe085f34d8..b5b2143292a6 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -93,6 +93,10 @@ ${project.version} test + + org.apache.spark + spark-test-tags_${scala.binary.version} + target/scala-${scala.binary.version}/classes diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index ffc4c32794ca..8f29fa91f7eb 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.scalatest.BeforeAndAfter import org.apache.spark.sql.SQLConf -import org.apache.spark.sql.hive.ExtendedHiveTest import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.tags.ExtendedHiveTest /** * Runs the test cases that are included in the hive distribution. diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 82cfeb2bb95d..d96f3e2b9f62 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -58,6 +58,10 @@ spark-sql_${scala.binary.version} ${project.version} + + org.apache.spark + spark-test-tags_${scala.binary.version} + diff --git a/tags/pom.xml b/tags/pom.xml new file mode 100644 index 000000000000..ca93722e7334 --- /dev/null +++ b/tags/pom.xml @@ -0,0 +1,50 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.10 + 1.6.0-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-test-tags_2.10 + jar + Spark Project Test Tags + http://spark.apache.org/ + + test-tags + + + + + org.scalatest + scalatest_${scala.binary.version} + compile + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/ExtendedHiveTest.java b/tags/src/main/java/org/apache/spark/tags/ExtendedHiveTest.java similarity index 96% rename from sql/hive/src/test/java/org/apache/spark/sql/hive/ExtendedHiveTest.java rename to tags/src/main/java/org/apache/spark/tags/ExtendedHiveTest.java index e2183183fb55..1b0c416b0fe4 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/ExtendedHiveTest.java +++ b/tags/src/main/java/org/apache/spark/tags/ExtendedHiveTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive; +package org.apache.spark.tags; import java.lang.annotation.*; import org.scalatest.TagAnnotation; diff --git a/yarn/src/test/java/org/apache/spark/deploy/yarn/ExtendedYarnTest.java b/tags/src/main/java/org/apache/spark/tags/ExtendedYarnTest.java similarity index 96% rename from yarn/src/test/java/org/apache/spark/deploy/yarn/ExtendedYarnTest.java rename to tags/src/main/java/org/apache/spark/tags/ExtendedYarnTest.java index 7a8f2fe979c1..2a631bfc88cf 100644 --- a/yarn/src/test/java/org/apache/spark/deploy/yarn/ExtendedYarnTest.java +++ b/tags/src/main/java/org/apache/spark/tags/ExtendedYarnTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.yarn; +package org.apache.spark.tags; import java.lang.annotation.*; import org.scalatest.TagAnnotation; diff --git a/unsafe/pom.xml b/unsafe/pom.xml index 4e8b9a84bb67..caf1f77890b5 100644 --- a/unsafe/pom.xml +++ b/unsafe/pom.xml @@ -55,6 +55,10 @@ + + org.apache.spark + spark-test-tags_${scala.binary.version} + org.mockito mockito-core diff --git a/yarn/pom.xml b/yarn/pom.xml index d8e4a4bbead8..3eadacba13e1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -51,6 +51,10 @@ test-jar test + + org.apache.spark + spark-test-tags_${scala.binary.version} + org.apache.hadoop hadoop-yarn-api diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 105c3090d489..f1601cd16100 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.launcher.TestClasspathBuilder import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.tags.ExtendedYarnTest import org.apache.spark.util.Utils /** diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 4700e2428df0..a85e5772a0fa 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -28,6 +28,7 @@ import org.scalatest.Matchers import org.apache.spark._ import org.apache.spark.network.shuffle.ShuffleTestAccessor import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} +import org.apache.spark.tags.ExtendedYarnTest /** * Integration test for the external shuffle service with a yarn mini-cluster From f3bb7b46288dbcfe3cc8554b084f38da7c20d3b4 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 15 Sep 2015 16:22:52 -0700 Subject: [PATCH 3/3] Disable mima for tags module. --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c97ef3b34ef8..1339980c3880 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -202,7 +202,7 @@ object SparkBuild extends PomBuild { (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) allProjects.filterNot(x => Seq(spark, hive, hiveThriftServer, catalyst, repl, - networkCommon, networkShuffle, networkYarn, unsafe).contains(x)).foreach { + networkCommon, networkShuffle, networkYarn, unsafe, testTags).contains(x)).foreach { x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) }