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 e31d90f60889..c0af98a04fb1 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
@@ -362,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/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..d65547e04db4 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.tags.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.tags.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-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 132062f94fb4..57f83607365d 100644
--- a/external/flume/pom.xml
+++ b/external/flume/pom.xml
@@ -67,14 +67,8 @@
test
- junit
- junit
- test
-
-
- com.novocode
- junit-interface
- test
+ org.apache.spark
+ spark-test-tags_${scala.binary.version}
diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml
index 05abd9e2e681..79258c126e04 100644
--- a/external/kafka/pom.xml
+++ b/external/kafka/pom.xml
@@ -87,14 +87,8 @@
test
- junit
- junit
- test
-
-
- com.novocode
- junit-interface
- test
+ org.apache.spark
+ spark-test-tags_${scala.binary.version}
diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml
index 05e6338a08b0..59fba8b826b4 100644
--- a/external/mqtt/pom.xml
+++ b/external/mqtt/pom.xml
@@ -58,22 +58,16 @@
scalacheck_${scala.binary.version}
test
-
- junit
- junit
- test
-
-
- com.novocode
- junit-interface
- test
-
org.apache.activemq
activemq-core
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 244ad58ae959..4c22ec8b3b15 100644
--- a/external/twitter/pom.xml
+++ b/external/twitter/pom.xml
@@ -59,14 +59,8 @@
test
- junit
- junit
- test
-
-
- com.novocode
- junit-interface
- test
+ org.apache.spark
+ spark-test-tags_${scala.binary.version}
diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml
index 171df8682c84..02d6b8128157 100644
--- a/external/zeromq/pom.xml
+++ b/external/zeromq/pom.xml
@@ -58,14 +58,8 @@
test
- junit
- junit
- test
-
-
- com.novocode
- junit-interface
- test
+ org.apache.spark
+ spark-test-tags_${scala.binary.version}
diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml
index 81794a853631..4ce90e75fd35 100644
--- a/extras/java8-tests/pom.xml
+++ b/extras/java8-tests/pom.xml
@@ -59,14 +59,8 @@
test
- junit
- junit
- test
-
-
- com.novocode
- junit-interface
- 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 6dd8ff69c294..ef72d97eae69 100644
--- a/extras/kinesis-asl/pom.xml
+++ b/extras/kinesis-asl/pom.xml
@@ -75,9 +75,8 @@
test
- com.novocode
- junit-interface
- test
+ org.apache.spark
+ spark-test-tags_${scala.binary.version}
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 ed38e66aa246..d595d74642ab 100644
--- a/launcher/pom.xml
+++ b/launcher/pom.xml
@@ -42,11 +42,6 @@
log4j
test
-
- junit
- junit
- test
-
org.mockito
mockito-core
@@ -63,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 22c0c6008ba3..70139121d8c7 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
@@ -131,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 1cc054a8936c..9af6cc5e925f 100644
--- a/network/common/pom.xml
+++ b/network/common/pom.xml
@@ -64,21 +64,15 @@
-
- junit
- junit
- test
-
-
- com.novocode
- junit-interface
- test
-
log4j
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 7a66c968041c..70ba5cb1995b 100644
--- a/network/shuffle/pom.xml
+++ b/network/shuffle/pom.xml
@@ -79,14 +79,8 @@
test
- junit
- junit
- test
-
-
- com.novocode
- junit-interface
- test
+ org.apache.spark
+ spark-test-tags_${scala.binary.version}
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 653599464114..530bc5dbbb5c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -86,6 +86,7 @@
+ tags
core
bagel
graphx
@@ -181,6 +182,7 @@
0.9.2
${java.home}
+
@@ -1952,6 +1971,7 @@
__not_used__
+ ${test.exclude.tags}
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 901cfa538d23..1339980c3880 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",
@@ -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)
}
@@ -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/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 465aa3a3888c..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
@@ -73,11 +77,6 @@
jackson-databind
${fasterxml.jackson.version}
-
- junit
- junit
- test
-
org.scalacheck
scalacheck_${scala.binary.version}
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 ab309e0a1d36..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
@@ -25,10 +25,12 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark.sql.SQLConf
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.
*/
+@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..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}
+
@@ -84,21 +88,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/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/tags/src/main/java/org/apache/spark/tags/ExtendedHiveTest.java b/tags/src/main/java/org/apache/spark/tags/ExtendedHiveTest.java
new file mode 100644
index 000000000000..1b0c416b0fe4
--- /dev/null
+++ b/tags/src/main/java/org/apache/spark/tags/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.tags;
+
+import java.lang.annotation.*;
+import org.scalatest.TagAnnotation;
+
+@TagAnnotation
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD, ElementType.TYPE})
+public @interface ExtendedHiveTest { }
diff --git a/tags/src/main/java/org/apache/spark/tags/ExtendedYarnTest.java b/tags/src/main/java/org/apache/spark/tags/ExtendedYarnTest.java
new file mode 100644
index 000000000000..2a631bfc88cf
--- /dev/null
+++ b/tags/src/main/java/org/apache/spark/tags/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.tags;
+
+import java.lang.annotation.*;
+import org.scalatest.TagAnnotation;
+
+@TagAnnotation
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD, ElementType.TYPE})
+public @interface ExtendedYarnTest { }
diff --git a/unsafe/pom.xml b/unsafe/pom.xml
index 066abe92e51c..caf1f77890b5 100644
--- a/unsafe/pom.xml
+++ b/unsafe/pom.xml
@@ -56,14 +56,8 @@
- junit
- junit
- test
-
-
- com.novocode
- junit-interface
- test
+ org.apache.spark
+ spark-test-tags_${scala.binary.version}
org.mockito
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 b5a42fd6afd9..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
/**
@@ -39,6 +40,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..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,10 +28,12 @@ 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
*/
+@ExtendedYarnTest
class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite {
override def newYarnConfig(): YarnConfiguration = {