@@ -132,7 +133,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
val totalTasksNumStr = if (totalTasks == storedTasks) {
s"$totalTasks"
} else {
- s"$storedTasks, showing ${totalTasks}"
+ s"$totalTasks, showing $storedTasks"
}
val summary =
@@ -685,7 +686,7 @@ private[ui] class TaskDataSource(
private var _tasksToShow: Seq[TaskData] = null
- override def dataSize: Int = taskCount(stage)
+ override def dataSize: Int = store.taskCount(stage.stageId, stage.attemptId).toInt
override def sliceData(from: Int, to: Int): Seq[TaskData] = {
if (_tasksToShow == null) {
@@ -847,7 +848,7 @@ private[ui] class TaskPagedTable(
{rdd.id}
-
{rdd.name}
diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
index b6c300c4778b1..43d62561e8eba 100644
--- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
@@ -175,7 +175,7 @@ private[spark] object ClosureCleaner extends Logging {
closure.getClass.isSynthetic &&
closure
.getClass
- .getInterfaces.exists(_.getName.equals("scala.Serializable"))
+ .getInterfaces.exists(_.getName == "scala.Serializable")
if (isClosureCandidate) {
try {
diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
index 21acaa95c5645..f4d6c7a28d2e4 100644
--- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
+++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
@@ -25,11 +25,14 @@ private[spark]
abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterator[A] {
private[this] var completed = false
- def next(): A = sub.next()
+ private[this] var iter = sub
+ def next(): A = iter.next()
def hasNext: Boolean = {
- val r = sub.hasNext
+ val r = iter.hasNext
if (!r && !completed) {
completed = true
+ // reassign to release resources of highly resource consuming iterators early
+ iter = Iterator.empty.asInstanceOf[I]
completion()
}
r
diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala
deleted file mode 100644
index 034826c57ef1d..0000000000000
--- a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.util
-
-import java.net.{URL, URLClassLoader}
-import java.util.Enumeration
-
-import scala.collection.JavaConverters._
-
-/**
- * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader.
- */
-private[spark] class MutableURLClassLoader(urls: Array[URL], parent: ClassLoader)
- extends URLClassLoader(urls, parent) {
-
- override def addURL(url: URL): Unit = {
- super.addURL(url)
- }
-
- override def getURLs(): Array[URL] = {
- super.getURLs()
- }
-
-}
-
-/**
- * A mutable class loader that gives preference to its own URLs over the parent class loader
- * when loading classes and resources.
- */
-private[spark] class ChildFirstURLClassLoader(urls: Array[URL], parent: ClassLoader)
- extends MutableURLClassLoader(urls, null) {
-
- private val parentClassLoader = new ParentClassLoader(parent)
-
- override def loadClass(name: String, resolve: Boolean): Class[_] = {
- try {
- super.loadClass(name, resolve)
- } catch {
- case e: ClassNotFoundException =>
- parentClassLoader.loadClass(name, resolve)
- }
- }
-
- override def getResource(name: String): URL = {
- val url = super.findResource(name)
- val res = if (url != null) url else parentClassLoader.getResource(name)
- res
- }
-
- override def getResources(name: String): Enumeration[URL] = {
- val childUrls = super.findResources(name).asScala
- val parentUrls = parentClassLoader.getResources(name).asScala
- (childUrls ++ parentUrls).asJavaEnumeration
- }
-
- override def addURL(url: URL) {
- super.addURL(url)
- }
-
-}
diff --git a/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala b/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala
index ce06e18879a49..c105f3229af09 100644
--- a/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala
+++ b/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala
@@ -100,7 +100,7 @@ private[spark] abstract class PeriodicCheckpointer[T](
var canDelete = true
while (checkpointQueue.size > 1 && canDelete) {
// Delete the oldest checkpoint only if the next checkpoint exists.
- if (isCheckpointed(checkpointQueue.head)) {
+ if (isCheckpointed(checkpointQueue(1))) {
removeCheckpointFile()
} else {
canDelete = false
diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
index 3bfdf95db84c6..8212cb931db52 100644
--- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
@@ -33,7 +33,7 @@ import org.apache.spark.util.collection.OpenHashSet
/**
* A trait that allows a class to give [[SizeEstimator]] more accurate size estimation.
* When a class extends it, [[SizeEstimator]] will query the `estimatedSize` first.
- * If `estimatedSize` does not return [[None]], [[SizeEstimator]] will use the returned size
+ * If `estimatedSize` does not return `None`, [[SizeEstimator]] will use the returned size
* as the size of the object. Otherwise, [[SizeEstimator]] will do the estimation work.
* The difference between a [[KnownSizeEstimation]] and
* [[org.apache.spark.util.collection.SizeTracker]] is that, a
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 15c958d3f511e..8f86b472b9373 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -19,7 +19,6 @@ package org.apache.spark.util
import java.io._
import java.lang.{Byte => JByte}
-import java.lang.InternalError
import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo}
import java.lang.reflect.InvocationTargetException
import java.math.{MathContext, RoundingMode}
@@ -240,6 +239,19 @@ private[spark] object Utils extends Logging {
// scalastyle:on classforname
}
+ /**
+ * Run a segment of code using a different context class loader in the current thread
+ */
+ def withContextClassLoader[T](ctxClassLoader: ClassLoader)(fn: => T): T = {
+ val oldClassLoader = Thread.currentThread().getContextClassLoader()
+ try {
+ Thread.currentThread().setContextClassLoader(ctxClassLoader)
+ fn
+ } finally {
+ Thread.currentThread().setContextClassLoader(oldClassLoader)
+ }
+ }
+
/**
* Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.DataOutput]]
*/
@@ -1073,7 +1085,7 @@ private[spark] object Utils extends Logging {
}
/**
- * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If
+ * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If
* no suffix is provided, the passed number is assumed to be in ms.
*/
def timeStringAsMs(str: String): Long = {
@@ -2052,6 +2064,30 @@ private[spark] object Utils extends Logging {
}
}
+ /**
+ * Implements the same logic as JDK `java.lang.String#trim` by removing leading and trailing
+ * non-printable characters less or equal to '\u0020' (SPACE) but preserves natural line
+ * delimiters according to [[java.util.Properties]] load method. The natural line delimiters are
+ * removed by JDK during load. Therefore any remaining ones have been specifically provided and
+ * escaped by the user, and must not be ignored
+ *
+ * @param str
+ * @return the trimmed value of str
+ */
+ private[util] def trimExceptCRLF(str: String): String = {
+ val nonSpaceOrNaturalLineDelimiter: Char => Boolean = { ch =>
+ ch > ' ' || ch == '\r' || ch == '\n'
+ }
+
+ val firstPos = str.indexWhere(nonSpaceOrNaturalLineDelimiter)
+ val lastPos = str.lastIndexWhere(nonSpaceOrNaturalLineDelimiter)
+ if (firstPos >= 0 && lastPos >= 0) {
+ str.substring(firstPos, lastPos + 1)
+ } else {
+ ""
+ }
+ }
+
/** Load properties present in the given file. */
def getPropertiesFromFile(filename: String): Map[String, String] = {
val file = new File(filename)
@@ -2062,8 +2098,10 @@ private[spark] object Utils extends Logging {
try {
val properties = new Properties()
properties.load(inReader)
- properties.stringPropertyNames().asScala.map(
- k => (k, properties.getProperty(k).trim)).toMap
+ properties.stringPropertyNames().asScala
+ .map { k => (k, trimExceptCRLF(properties.getProperty(k))) }
+ .toMap
+
} catch {
case e: IOException =>
throw new SparkException(s"Failed when loading Spark properties from $filename", e)
@@ -2795,6 +2833,36 @@ private[spark] object Utils extends Logging {
}
}
}
+
+ /**
+ * Regular expression matching full width characters.
+ *
+ * Looked at all the 0x0000-0xFFFF characters (unicode) and showed them under Xshell.
+ * Found all the full width characters, then get the regular expression.
+ */
+ private val fullWidthRegex = ("""[""" +
+ // scalastyle:off nonascii
+ """\u1100-\u115F""" +
+ """\u2E80-\uA4CF""" +
+ """\uAC00-\uD7A3""" +
+ """\uF900-\uFAFF""" +
+ """\uFE10-\uFE19""" +
+ """\uFE30-\uFE6F""" +
+ """\uFF00-\uFF60""" +
+ """\uFFE0-\uFFE6""" +
+ // scalastyle:on nonascii
+ """]""").r
+
+ /**
+ * Return the number of half widths in a given string. Note that a full width character
+ * occupies two half widths.
+ *
+ * For a string consisting of 1 million characters, the execution of this method requires
+ * about 50ms.
+ */
+ def stringHalfWidth(str: String): Int = {
+ if (str == null) 0 else str.length + fullWidthRegex.findAllIn(str).size
+ }
}
private[util] object CallerContext extends Logging {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
index b159200d79222..547a862467c88 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
@@ -727,9 +727,10 @@ private[spark] class ExternalSorter[K, V, C](
spills.clear()
forceSpillFiles.foreach(s => s.file.delete())
forceSpillFiles.clear()
- if (map != null || buffer != null) {
+ if (map != null || buffer != null || readingIterator != null) {
map = null // So that the memory can be garbage-collected
buffer = null // So that the memory can be garbage-collected
+ readingIterator = null // So that the memory can be garbage-collected
releaseMemory()
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
index 39f050f6ca5ad..870830fff4c3e 100644
--- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
@@ -19,18 +19,18 @@ package org.apache.spark.util.io
import java.io.{File, FileInputStream, InputStream}
import java.nio.ByteBuffer
-import java.nio.channels.{FileChannel, WritableByteChannel}
-import java.nio.file.StandardOpenOption
-
-import scala.collection.mutable.ListBuffer
+import java.nio.channels.WritableByteChannel
+import com.google.common.io.ByteStreams
import com.google.common.primitives.UnsignedBytes
+import org.apache.commons.io.IOUtils
import org.apache.spark.SparkEnv
import org.apache.spark.internal.config
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
-import org.apache.spark.network.util.ByteArrayWritableChannel
-import org.apache.spark.storage.StorageUtils
+import org.apache.spark.network.util.{ByteArrayWritableChannel, LimitedInputStream}
+import org.apache.spark.storage.{EncryptedManagedBuffer, StorageUtils}
+import org.apache.spark.unsafe.array.ByteArrayMethods
import org.apache.spark.util.Utils
/**
@@ -97,7 +97,7 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
* @throws UnsupportedOperationException if this buffer's size exceeds the maximum array size.
*/
def toArray: Array[Byte] = {
- if (size >= Integer.MAX_VALUE) {
+ if (size >= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) {
throw new UnsupportedOperationException(
s"cannot call toArray because buffer size ($size bytes) exceeds maximum array size")
}
@@ -170,35 +170,44 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
}
-object ChunkedByteBuffer {
- // TODO eliminate this method if we switch BlockManager to getting InputStreams
- def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = {
+private[spark] object ChunkedByteBuffer {
+
+
+ // TODO SPARK-25905 eliminate this method if we switch BlockManager to getting InputStreams
+ def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = {
data match {
case f: FileSegmentManagedBuffer =>
- map(f.getFile, maxChunkSize, f.getOffset, f.getLength)
+ fromFile(f.getFile, f.getOffset, f.getLength)
+ case e: EncryptedManagedBuffer =>
+ e.blockData.toChunkedByteBuffer(ByteBuffer.allocate _)
case other =>
new ChunkedByteBuffer(other.nioByteBuffer())
}
}
- def map(file: File, maxChunkSize: Int): ChunkedByteBuffer = {
- map(file, maxChunkSize, 0, file.length())
+ def fromFile(file: File): ChunkedByteBuffer = {
+ fromFile(file, 0, file.length())
}
- def map(file: File, maxChunkSize: Int, offset: Long, length: Long): ChunkedByteBuffer = {
- Utils.tryWithResource(FileChannel.open(file.toPath, StandardOpenOption.READ)) { channel =>
- var remaining = length
- var pos = offset
- val chunks = new ListBuffer[ByteBuffer]()
- while (remaining > 0) {
- val chunkSize = math.min(remaining, maxChunkSize)
- val chunk = channel.map(FileChannel.MapMode.READ_ONLY, pos, chunkSize)
- pos += chunkSize
- remaining -= chunkSize
- chunks += chunk
- }
- new ChunkedByteBuffer(chunks.toArray)
+ private def fromFile(
+ file: File,
+ offset: Long,
+ length: Long): ChunkedByteBuffer = {
+ // We do *not* memory map the file, because we may end up putting this into the memory store,
+ // and spark currently is not expecting memory-mapped buffers in the memory store, it conflicts
+ // with other parts that manage the lifecyle of buffers and dispose them. See SPARK-25422.
+ val is = new FileInputStream(file)
+ ByteStreams.skipFully(is, offset)
+ val in = new LimitedInputStream(is, length)
+ val chunkSize = math.min(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, length).toInt
+ val out = new ChunkedByteBufferOutputStream(chunkSize, ByteBuffer.allocate _)
+ Utils.tryWithSafeFinally {
+ IOUtils.copy(in, out)
+ } {
+ in.close()
+ out.close()
}
+ out.toChunkedByteBuffer
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
index e8cdb6e98bf36..67ad513006649 100644
--- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
@@ -62,7 +62,7 @@ private[spark] object XORShiftRandom {
/** Hash seeds to have 0/1 bits throughout. */
private[random] def hashSeed(seed: Long): Long = {
val bytes = ByteBuffer.allocate(java.lang.Long.SIZE).putLong(seed).array()
- val lowBits = MurmurHash3.bytesHash(bytes)
+ val lowBits = MurmurHash3.bytesHash(bytes, MurmurHash3.arraySeed)
val highBits = MurmurHash3.bytesHash(bytes, lowBits)
(highBits.toLong << 32) | (lowBits.toLong & 0xFFFFFFFFL)
}
diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java
new file mode 100644
index 0000000000000..80cd70282a51d
--- /dev/null
+++ b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java
@@ -0,0 +1,139 @@
+/*
+ * 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;
+
+import org.apache.spark.api.java.JavaSparkContext;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class ExecutorPluginSuite {
+ private static final String EXECUTOR_PLUGIN_CONF_NAME = "spark.executor.plugins";
+ private static final String testBadPluginName = TestBadShutdownPlugin.class.getName();
+ private static final String testPluginName = TestExecutorPlugin.class.getName();
+ private static final String testSecondPluginName = TestSecondPlugin.class.getName();
+
+ // Static value modified by testing plugins to ensure plugins loaded correctly.
+ public static int numSuccessfulPlugins = 0;
+
+ // Static value modified by testing plugins to verify plugins shut down properly.
+ public static int numSuccessfulTerminations = 0;
+
+ private JavaSparkContext sc;
+
+ @Before
+ public void setUp() {
+ sc = null;
+ numSuccessfulPlugins = 0;
+ numSuccessfulTerminations = 0;
+ }
+
+ @After
+ public void tearDown() {
+ if (sc != null) {
+ sc.stop();
+ sc = null;
+ }
+ }
+
+ private SparkConf initializeSparkConf(String pluginNames) {
+ return new SparkConf()
+ .setMaster("local")
+ .setAppName("test")
+ .set(EXECUTOR_PLUGIN_CONF_NAME, pluginNames);
+ }
+
+ @Test
+ public void testPluginClassDoesNotExist() {
+ SparkConf conf = initializeSparkConf("nonexistent.plugin");
+ try {
+ sc = new JavaSparkContext(conf);
+ fail("No exception thrown for nonexistent plugin");
+ } catch (Exception e) {
+ // We cannot catch ClassNotFoundException directly because Java doesn't think it'll be thrown
+ assertTrue(e.toString().startsWith("java.lang.ClassNotFoundException"));
+ }
+ }
+
+ @Test
+ public void testAddPlugin() throws InterruptedException {
+ // Load the sample TestExecutorPlugin, which will change the value of numSuccessfulPlugins
+ SparkConf conf = initializeSparkConf(testPluginName);
+ sc = new JavaSparkContext(conf);
+ assertEquals(1, numSuccessfulPlugins);
+ sc.stop();
+ sc = null;
+ assertEquals(1, numSuccessfulTerminations);
+ }
+
+ @Test
+ public void testAddMultiplePlugins() throws InterruptedException {
+ // Load two plugins and verify they both execute.
+ SparkConf conf = initializeSparkConf(testPluginName + "," + testSecondPluginName);
+ sc = new JavaSparkContext(conf);
+ assertEquals(2, numSuccessfulPlugins);
+ sc.stop();
+ sc = null;
+ assertEquals(2, numSuccessfulTerminations);
+ }
+
+ @Test
+ public void testPluginShutdownWithException() {
+ // Verify an exception in one plugin shutdown does not affect the others
+ String pluginNames = testPluginName + "," + testBadPluginName + "," + testPluginName;
+ SparkConf conf = initializeSparkConf(pluginNames);
+ sc = new JavaSparkContext(conf);
+ assertEquals(3, numSuccessfulPlugins);
+ sc.stop();
+ sc = null;
+ assertEquals(2, numSuccessfulTerminations);
+ }
+
+ public static class TestExecutorPlugin implements ExecutorPlugin {
+ public void init() {
+ ExecutorPluginSuite.numSuccessfulPlugins++;
+ }
+
+ public void shutdown() {
+ ExecutorPluginSuite.numSuccessfulTerminations++;
+ }
+ }
+
+ public static class TestSecondPlugin implements ExecutorPlugin {
+ public void init() {
+ ExecutorPluginSuite.numSuccessfulPlugins++;
+ }
+
+ public void shutdown() {
+ ExecutorPluginSuite.numSuccessfulTerminations++;
+ }
+ }
+
+ public static class TestBadShutdownPlugin implements ExecutorPlugin {
+ public void init() {
+ ExecutorPluginSuite.numSuccessfulPlugins++;
+ }
+
+ public void shutdown() {
+ throw new RuntimeException("This plugin will fail to cleanly shut down");
+ }
+ }
+}
diff --git a/core/src/test/java/org/apache/spark/io/GenericFileInputStreamSuite.java b/core/src/test/java/org/apache/spark/io/GenericFileInputStreamSuite.java
index 22db3592ecc96..8ff787975eaae 100644
--- a/core/src/test/java/org/apache/spark/io/GenericFileInputStreamSuite.java
+++ b/core/src/test/java/org/apache/spark/io/GenericFileInputStreamSuite.java
@@ -48,8 +48,12 @@ public void setUp() throws IOException {
}
@After
- public void tearDown() {
+ public void tearDown() throws IOException {
inputFile.delete();
+
+ for (InputStream is : inputStreams) {
+ is.close();
+ }
}
@Test
@@ -141,4 +145,15 @@ public void testBytesSkippedAfterEOF() throws IOException {
assertEquals(-1, inputStream.read());
}
}
+
+ @Test
+ public void testReadPastEOF() throws IOException {
+ InputStream is = inputStreams[0];
+ byte[] buf = new byte[1024];
+ int read;
+ while ((read = is.read(buf, 0, buf.length)) != -1);
+
+ int readAfterEOF = is.read(buf, 0, buf.length);
+ assertEquals(-1, readAfterEOF);
+ }
}
diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
index d7d2d0b012bd3..a0664b30d6cc2 100644
--- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
+++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
@@ -76,7 +76,7 @@ public void freeingPageSetsPageNumberToSpecialConstant() {
final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP);
final MemoryBlock dataPage = manager.allocatePage(256, c);
c.freePage(dataPage);
- Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.getPageNumber());
+ Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.pageNumber);
}
@Test(expected = AssertionError.class)
diff --git a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java
index 0bbaea6b834b8..6aa577d1bf797 100644
--- a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java
+++ b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java
@@ -38,12 +38,12 @@ public long spill(long size, MemoryConsumer trigger) throws IOException {
return used;
}
- void use(long size) {
+ public void use(long size) {
long got = taskMemoryManager.acquireExecutionMemory(size, this);
used += got;
}
- void free(long size) {
+ public void free(long size) {
used -= size;
taskMemoryManager.releaseExecutionMemory(size, this);
}
diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
index faa70f23b0ac6..0d5c5ea7903e9 100644
--- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
@@ -233,7 +233,6 @@ public void writeEmptyIterator() throws Exception {
writer.write(Iterators.emptyIterator());
final Option mapStatus = writer.stop(true);
assertTrue(mapStatus.isDefined());
- assertEquals(0, mapStatus.get().numberOfOutput());
assertTrue(mergedOutputFile.exists());
assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile);
assertEquals(0, taskMetrics.shuffleWriteMetrics().recordsWritten());
@@ -253,7 +252,6 @@ public void writeWithoutSpilling() throws Exception {
writer.write(dataToWrite.iterator());
final Option mapStatus = writer.stop(true);
assertTrue(mapStatus.isDefined());
- assertEquals(NUM_PARTITITONS, mapStatus.get().numberOfOutput());
assertTrue(mergedOutputFile.exists());
long sumOfPartitionSizes = 0;
diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
index 53a233f698c7a..77416549c9b14 100644
--- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
+++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
@@ -33,6 +33,9 @@
import org.apache.spark.SparkConf;
import org.apache.spark.executor.ShuffleWriteMetrics;
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.memory.SparkOutOfMemoryError;
+import org.apache.spark.memory.TestMemoryConsumer;
import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.memory.TestMemoryManager;
import org.apache.spark.network.util.JavaUtils;
@@ -667,4 +670,67 @@ public void testPeakMemoryUsed() {
}
}
+ @Test
+ public void avoidDeadlock() throws InterruptedException {
+ memoryManager.limit(PAGE_SIZE_BYTES);
+ MemoryMode mode = useOffHeapMemoryAllocator() ? MemoryMode.OFF_HEAP: MemoryMode.ON_HEAP;
+ TestMemoryConsumer c1 = new TestMemoryConsumer(taskMemoryManager, mode);
+ BytesToBytesMap map =
+ new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024, false);
+
+ Thread thread = new Thread(() -> {
+ int i = 0;
+ long used = 0;
+ while (i < 10) {
+ c1.use(10000000);
+ used += 10000000;
+ i++;
+ }
+ c1.free(used);
+ });
+
+ try {
+ int i;
+ for (i = 0; i < 1024; i++) {
+ final long[] arr = new long[]{i};
+ final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8);
+ loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8);
+ }
+
+ // Starts to require memory at another memory consumer.
+ thread.start();
+
+ BytesToBytesMap.MapIterator iter = map.destructiveIterator();
+ for (i = 0; i < 1024; i++) {
+ iter.next();
+ }
+ assertFalse(iter.hasNext());
+ } finally {
+ map.free();
+ thread.join();
+ for (File spillFile : spillFilesCreated) {
+ assertFalse("Spill file " + spillFile.getPath() + " was not cleaned up",
+ spillFile.exists());
+ }
+ }
+ }
+
+ @Test
+ public void freeAfterFailedReset() {
+ // SPARK-29244: BytesToBytesMap.free after a OOM reset operation should not cause failure.
+ memoryManager.limit(5000);
+ BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager,
+ 256, 0.5, 4000, false);
+ // Force OOM on next memory allocation.
+ memoryManager.markExecutionAsOutOfMemoryOnce();
+ try {
+ map.reset();
+ Assert.fail("Expected SparkOutOfMemoryError to be thrown");
+ } catch (SparkOutOfMemoryError e) {
+ // Expected exception; do nothing.
+ } finally {
+ map.free();
+ }
+ }
+
}
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index 629a323042ff2..9398d5e74f1ca 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -336,6 +336,21 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
}
}
+ test("reference partitions inside a task") {
+ // Run a simple job which just makes sure there is no failure if we touch rdd.partitions
+ // inside a task. This requires the stateLock to be serializable. This is very convoluted
+ // use case, it's just a check for backwards-compatibility after the fix for SPARK-28917.
+ sc = new SparkContext("local-cluster[1,1,1024]", "test")
+ val rdd1 = sc.parallelize(1 to 10, 1)
+ val rdd2 = rdd1.map { x => x + 1}
+ // ensure we can force computation of rdd2.dependencies inside a task. Just touching
+ // it will force computation and touching the stateLock. The check for null is to just
+ // to make sure that we've setup our test correctly, and haven't precomputed dependencies
+ // in the driver
+ val dependencyComputeCount = rdd1.map { x => if (rdd2.dependencies == null) 1 else 0}.sum()
+ assert(dependencyComputeCount > 0)
+ }
+
}
object DistributedSuite {
diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
index 5c718cb654ce8..df5d2658e8c7c 100644
--- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
@@ -21,7 +21,7 @@ import scala.collection.mutable
import org.mockito.Matchers.{any, eq => meq}
import org.mockito.Mockito.{mock, never, verify, when}
-import org.scalatest.{BeforeAndAfter, PrivateMethodTester}
+import org.scalatest.PrivateMethodTester
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.internal.config
@@ -37,20 +37,24 @@ import org.apache.spark.util.ManualClock
*/
class ExecutorAllocationManagerSuite
extends SparkFunSuite
- with LocalSparkContext
- with BeforeAndAfter {
+ with LocalSparkContext {
import ExecutorAllocationManager._
import ExecutorAllocationManagerSuite._
private val contexts = new mutable.ListBuffer[SparkContext]()
- before {
+ override def beforeEach(): Unit = {
+ super.beforeEach()
contexts.clear()
}
- after {
- contexts.foreach(_.stop())
+ override def afterEach(): Unit = {
+ try {
+ contexts.foreach(_.stop())
+ } finally {
+ super.afterEach()
+ }
}
private def post(bus: LiveListenerBus, event: SparkListenerEvent): Unit = {
@@ -281,7 +285,7 @@ class ExecutorAllocationManagerSuite
assert(totalRunningTasks(manager) === 0)
}
- test("cancel pending executors when no longer needed") {
+ testRetry("cancel pending executors when no longer needed") {
sc = createSparkContext(0, 10, 0)
val manager = sc.executorAllocationManager.get
post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(2, 5)))
@@ -420,6 +424,7 @@ class ExecutorAllocationManagerSuite
// Remove when numExecutorsTarget is the same as the current number of executors
assert(addExecutors(manager) === 1)
assert(addExecutors(manager) === 2)
+ (1 to 8).foreach(execId => onExecutorAdded(manager, execId.toString))
(1 to 8).map { i => createTaskInfo(i, i, s"$i") }.foreach {
info => post(sc.listenerBus, SparkListenerTaskStart(0, 0, info)) }
assert(executorIds(manager).size === 8)
@@ -833,7 +838,7 @@ class ExecutorAllocationManagerSuite
assert(removeTimes(manager).size === 1)
}
- test("SPARK-4951: call onTaskStart before onBlockManagerAdded") {
+ test("SPARK-4951: call onTaskStart before onExecutorAdded") {
sc = createSparkContext(2, 10, 2)
val manager = sc.executorAllocationManager.get
assert(executorIds(manager).isEmpty)
@@ -935,12 +940,7 @@ class ExecutorAllocationManagerSuite
assert(maxNumExecutorsNeeded(manager) === 0)
schedule(manager)
- // Verify executor is timeout but numExecutorsTarget is not recalculated
- assert(numExecutorsTarget(manager) === 3)
-
- // Schedule again to recalculate the numExecutorsTarget after executor is timeout
- schedule(manager)
- // Verify that current number of executors should be ramp down when executor is timeout
+ // Verify executor is timeout,numExecutorsTarget is recalculated
assert(numExecutorsTarget(manager) === 2)
}
@@ -1147,6 +1147,48 @@ class ExecutorAllocationManagerSuite
verify(mockAllocationClient).killExecutors(Seq("executor-1"), false, false, false)
}
+ test("SPARK-26758 check executor target number after idle time out ") {
+ sc = createSparkContext(1, 5, 3)
+ val manager = sc.executorAllocationManager.get
+ val clock = new ManualClock(10000L)
+ manager.setClock(clock)
+ assert(numExecutorsTarget(manager) === 3)
+ manager.listener.onExecutorAdded(SparkListenerExecutorAdded(
+ clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty)))
+ manager.listener.onExecutorAdded(SparkListenerExecutorAdded(
+ clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 2, Map.empty)))
+ manager.listener.onExecutorAdded(SparkListenerExecutorAdded(
+ clock.getTimeMillis(), "executor-3", new ExecutorInfo("host1", 3, Map.empty)))
+ // make all the executors as idle, so that it will be killed
+ clock.advance(executorIdleTimeout * 1000)
+ schedule(manager)
+ // once the schedule is run target executor number should be 1
+ assert(numExecutorsTarget(manager) === 1)
+ }
+
+ test("SPARK-26927 call onExecutorRemoved before onTaskStart") {
+ sc = createSparkContext(2, 5)
+ val manager = sc.executorAllocationManager.get
+ assert(executorIds(manager).isEmpty)
+ post(sc.listenerBus, SparkListenerExecutorAdded(
+ 0L, "1", new ExecutorInfo("host1", 1, Map.empty)))
+ post(sc.listenerBus, SparkListenerExecutorAdded(
+ 0L, "2", new ExecutorInfo("host2", 1, Map.empty)))
+ post(sc.listenerBus, SparkListenerExecutorAdded(
+ 0L, "3", new ExecutorInfo("host3", 1, Map.empty)))
+ assert(executorIds(manager).size === 3)
+
+ post(sc.listenerBus, SparkListenerExecutorRemoved(0L, "3", "disconnected"))
+ assert(executorIds(manager).size === 2)
+ assert(executorIds(manager) === Set("1", "2"))
+
+ val taskInfo1 = createTaskInfo(0, 0, "3")
+ post(sc.listenerBus, SparkListenerTaskStart(0, 0, taskInfo1))
+ // Verify taskStart not adding already removed executors.
+ assert(executorIds(manager).size === 2)
+ assert(executorIds(manager) === Set("1", "2"))
+ }
+
private def createSparkContext(
minExecutors: Int = 1,
maxExecutors: Int = 5,
diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
index a441b9c8ab97a..34efcdf4bc886 100644
--- a/core/src/test/scala/org/apache/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -19,10 +19,12 @@ package org.apache.spark
import java.io._
import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
import java.util.zip.GZIPOutputStream
import scala.io.Source
+import com.google.common.io.Files
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io._
@@ -299,6 +301,38 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
}
}
+ test("SPARK-22357 test binaryFiles minPartitions") {
+ sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")
+ .set("spark.files.openCostInBytes", "0")
+ .set("spark.default.parallelism", "1"))
+
+ val tempDir = Utils.createTempDir()
+ val tempDirPath = tempDir.getAbsolutePath
+
+ for (i <- 0 until 8) {
+ val tempFile = new File(tempDir, s"part-0000$i")
+ Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", tempFile,
+ StandardCharsets.UTF_8)
+ }
+
+ for (p <- Seq(1, 2, 8)) {
+ assert(sc.binaryFiles(tempDirPath, minPartitions = p).getNumPartitions === p)
+ }
+ }
+
+ test("minimum split size per node and per rack should be less than or equal to maxSplitSize") {
+ sc = new SparkContext("local", "test")
+ val testOutput = Array[Byte](1, 2, 3, 4, 5)
+ val outFile = writeBinaryData(testOutput, 1)
+ sc.hadoopConfiguration.setLong(
+ "mapreduce.input.fileinputformat.split.minsize.per.node", 5123456)
+ sc.hadoopConfiguration.setLong(
+ "mapreduce.input.fileinputformat.split.minsize.per.rack", 5123456)
+
+ val (_, data) = sc.binaryFiles(outFile.getAbsolutePath).collect().head
+ assert(data.toArray === testOutput)
+ }
+
test("fixed record length binary file as byte array") {
sc = new SparkContext("local", "test")
val testOutput = Array[Byte](1, 2, 3, 4, 5, 6)
diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
index 61da4138896cd..7b251c1d811b0 100644
--- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
@@ -363,7 +363,10 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
}.foreachAsync { x =>
// Block this code from being executed, until the job get cancelled. In this case, if the
// source iterator is interruptible, the max number of increment should be under
- // `numElements`.
+ // `numElements`. We sleep a little to make sure that we leave enough time for the
+ // "kill" message to be delivered to the executor (10000 * 10ms = 100s allowance for
+ // delivery, which should be more than enough).
+ Thread.sleep(10)
taskCancelledSemaphore.acquire()
executionOfInterruptibleCounter.getAndIncrement()
}
diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index e79739692fe13..21f481d477242 100644
--- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
@@ -62,9 +62,9 @@ class MapOutputTrackerSuite extends SparkFunSuite {
val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L))
val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L))
tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000),
- Array(1000L, 10000L), 10))
+ Array(1000L, 10000L)))
tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000),
- Array(10000L, 1000L), 10))
+ Array(10000L, 1000L)))
val statuses = tracker.getMapSizesByExecutorId(10, 0)
assert(statuses.toSet ===
Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))),
@@ -84,9 +84,9 @@ class MapOutputTrackerSuite extends SparkFunSuite {
val compressedSize1000 = MapStatus.compressSize(1000L)
val compressedSize10000 = MapStatus.compressSize(10000L)
tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000),
- Array(compressedSize1000, compressedSize10000), 10))
+ Array(compressedSize1000, compressedSize10000)))
tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000),
- Array(compressedSize10000, compressedSize1000), 10))
+ Array(compressedSize10000, compressedSize1000)))
assert(tracker.containsShuffle(10))
assert(tracker.getMapSizesByExecutorId(10, 0).nonEmpty)
assert(0 == tracker.getNumCachedSerializedBroadcast)
@@ -107,9 +107,9 @@ class MapOutputTrackerSuite extends SparkFunSuite {
val compressedSize1000 = MapStatus.compressSize(1000L)
val compressedSize10000 = MapStatus.compressSize(10000L)
tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000),
- Array(compressedSize1000, compressedSize1000, compressedSize1000), 10))
+ Array(compressedSize1000, compressedSize1000, compressedSize1000)))
tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000),
- Array(compressedSize10000, compressedSize1000, compressedSize1000), 10))
+ Array(compressedSize10000, compressedSize1000, compressedSize1000)))
assert(0 == tracker.getNumCachedSerializedBroadcast)
// As if we had two simultaneous fetch failures
@@ -145,7 +145,7 @@ class MapOutputTrackerSuite extends SparkFunSuite {
val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L))
masterTracker.registerMapOutput(10, 0, MapStatus(
- BlockManagerId("a", "hostA", 1000), Array(1000L), 10))
+ BlockManagerId("a", "hostA", 1000), Array(1000L)))
slaveTracker.updateEpoch(masterTracker.getEpoch)
assert(slaveTracker.getMapSizesByExecutorId(10, 0).toSeq ===
Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000)))))
@@ -182,7 +182,7 @@ class MapOutputTrackerSuite extends SparkFunSuite {
// Message size should be ~123B, and no exception should be thrown
masterTracker.registerShuffle(10, 1)
masterTracker.registerMapOutput(10, 0, MapStatus(
- BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0), 0))
+ BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0)))
val senderAddress = RpcAddress("localhost", 12345)
val rpcCallContext = mock(classOf[RpcCallContext])
when(rpcCallContext.senderAddress).thenReturn(senderAddress)
@@ -216,11 +216,11 @@ class MapOutputTrackerSuite extends SparkFunSuite {
// on hostB with output size 3
tracker.registerShuffle(10, 3)
tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000),
- Array(2L), 1))
+ Array(2L)))
tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("a", "hostA", 1000),
- Array(2L), 1))
+ Array(2L)))
tracker.registerMapOutput(10, 2, MapStatus(BlockManagerId("b", "hostB", 1000),
- Array(3L), 1))
+ Array(3L)))
// When the threshold is 50%, only host A should be returned as a preferred location
// as it has 4 out of 7 bytes of output.
@@ -260,7 +260,7 @@ class MapOutputTrackerSuite extends SparkFunSuite {
masterTracker.registerShuffle(20, 100)
(0 until 100).foreach { i =>
masterTracker.registerMapOutput(20, i, new CompressedMapStatus(
- BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0), 0))
+ BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0)))
}
val senderAddress = RpcAddress("localhost", 12345)
val rpcCallContext = mock(classOf[RpcCallContext])
@@ -309,9 +309,9 @@ class MapOutputTrackerSuite extends SparkFunSuite {
val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L))
val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L))
tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000),
- Array(size0, size1000, size0, size10000), 1))
+ Array(size0, size1000, size0, size10000)))
tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000),
- Array(size10000, size0, size1000, size0), 1))
+ Array(size10000, size0, size1000, size0)))
assert(tracker.containsShuffle(10))
assert(tracker.getMapSizesByExecutorId(10, 0, 4).toSeq ===
Seq(
diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
index 456f97b535ef6..b917469e48747 100644
--- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
@@ -391,7 +391,6 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC
assert(mapOutput2.isDefined)
assert(mapOutput1.get.location === mapOutput2.get.location)
assert(mapOutput1.get.getSizeForBlock(0) === mapOutput1.get.getSizeForBlock(0))
- assert(mapOutput1.get.numberOfOutput === mapOutput2.get.numberOfOutput)
// register one of the map outputs -- doesn't matter which one
mapOutput1.foreach { case mapStatus =>
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
index 0d06b02e74e34..66462de5588a6 100644
--- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -30,7 +30,7 @@ import org.apache.spark.deploy.history.config._
import org.apache.spark.internal.config._
import org.apache.spark.network.util.ByteUnit
import org.apache.spark.serializer.{JavaSerializer, KryoRegistrator, KryoSerializer}
-import org.apache.spark.util.{ResetSystemProperties, RpcUtils}
+import org.apache.spark.util.{ResetSystemProperties, RpcUtils, Utils}
class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSystemProperties {
test("Test byteString conversion") {
@@ -339,6 +339,32 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst
}
}
+ test("SPARK-26998: SSL configuration not needed on executors") {
+ val conf = new SparkConf(false)
+ conf.set("spark.ssl.enabled", "true")
+ conf.set("spark.ssl.keyPassword", "password")
+ conf.set("spark.ssl.keyStorePassword", "password")
+ conf.set("spark.ssl.trustStorePassword", "password")
+
+ val filtered = conf.getAll.filter { case (k, _) => SparkConf.isExecutorStartupConf(k) }
+ assert(filtered.isEmpty)
+ }
+
+ test("SPARK-27244 toDebugString redacts sensitive information") {
+ val conf = new SparkConf(loadDefaults = false)
+ .set("dummy.password", "dummy-password")
+ .set("spark.hadoop.hive.server2.keystore.password", "1234")
+ .set("spark.hadoop.javax.jdo.option.ConnectionPassword", "1234")
+ .set("spark.regular.property", "regular_value")
+ assert(conf.toDebugString ==
+ s"""
+ |dummy.password=${Utils.REDACTION_REPLACEMENT_TEXT}
+ |spark.hadoop.hive.server2.keystore.password=${Utils.REDACTION_REPLACEMENT_TEXT}
+ |spark.hadoop.javax.jdo.option.ConnectionPassword=${Utils.REDACTION_REPLACEMENT_TEXT}
+ |spark.regular.property=regular_value
+ """.stripMargin.trim)
+ }
+
val defaultIllegalValue = "SomeIllegalValue"
val illegalValueTests : Map[String, (SparkConf, String) => Any] = Map(
"getTimeAsSeconds" -> (_.getTimeAsSeconds(_)),
diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
index 8feb3dee050d2..c45f104d0aa95 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
@@ -17,7 +17,10 @@
package org.apache.spark
+import scala.concurrent.duration._
+
import org.scalatest.Assertions
+import org.scalatest.concurrent.Eventually._
import org.apache.spark.storage.StorageLevel
@@ -58,9 +61,12 @@ class SparkContextInfoSuite extends SparkFunSuite with LocalSparkContext {
test("getRDDStorageInfo only reports on RDDs that actually persist data") {
sc = new SparkContext("local", "test")
val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache()
- assert(sc.getRDDStorageInfo.size === 0)
+ assert(sc.getRDDStorageInfo.length === 0)
rdd.collect()
- assert(sc.getRDDStorageInfo.size === 1)
+ sc.listenerBus.waitUntilEmpty(10000)
+ eventually(timeout(10.seconds), interval(100.milliseconds)) {
+ assert(sc.getRDDStorageInfo.length === 1)
+ }
assert(sc.getRDDStorageInfo.head.isCached)
assert(sc.getRDDStorageInfo.head.memSize > 0)
assert(sc.getRDDStorageInfo.head.storageLevel === StorageLevel.MEMORY_ONLY)
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
index f8938dfedee5b..811b9757232e2 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
@@ -23,110 +23,129 @@ import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend
import org.apache.spark.scheduler.local.LocalSchedulerBackend
+import org.apache.spark.util.Utils
class SparkContextSchedulerCreationSuite
extends SparkFunSuite with LocalSparkContext with PrivateMethodTester with Logging {
- def createTaskScheduler(master: String): TaskSchedulerImpl =
- createTaskScheduler(master, "client")
+ def noOp(taskSchedulerImpl: TaskSchedulerImpl): Unit = {}
- def createTaskScheduler(master: String, deployMode: String): TaskSchedulerImpl =
- createTaskScheduler(master, deployMode, new SparkConf())
+ def createTaskScheduler(master: String)(body: TaskSchedulerImpl => Unit = noOp): Unit =
+ createTaskScheduler(master, "client")(body)
+
+ def createTaskScheduler(master: String, deployMode: String)(
+ body: TaskSchedulerImpl => Unit): Unit =
+ createTaskScheduler(master, deployMode, new SparkConf())(body)
def createTaskScheduler(
master: String,
deployMode: String,
- conf: SparkConf): TaskSchedulerImpl = {
+ conf: SparkConf)(body: TaskSchedulerImpl => Unit): Unit = {
// Create local SparkContext to setup a SparkEnv. We don't actually want to start() the
// real schedulers, so we don't want to create a full SparkContext with the desired scheduler.
sc = new SparkContext("local", "test", conf)
val createTaskSchedulerMethod =
PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler)
- val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, deployMode)
- sched.asInstanceOf[TaskSchedulerImpl]
+ val (_, sched) =
+ SparkContext invokePrivate createTaskSchedulerMethod(sc, master, deployMode)
+ try {
+ body(sched.asInstanceOf[TaskSchedulerImpl])
+ } finally {
+ Utils.tryLogNonFatalError {
+ sched.stop()
+ }
+ }
}
test("bad-master") {
val e = intercept[SparkException] {
- createTaskScheduler("localhost:1234")
+ createTaskScheduler("localhost:1234")()
}
assert(e.getMessage.contains("Could not parse Master URL"))
}
test("local") {
- val sched = createTaskScheduler("local")
- sched.backend match {
- case s: LocalSchedulerBackend => assert(s.totalCores === 1)
- case _ => fail()
+ val sched = createTaskScheduler("local") { sched =>
+ sched.backend match {
+ case s: LocalSchedulerBackend => assert(s.totalCores === 1)
+ case _ => fail()
+ }
}
}
test("local-*") {
- val sched = createTaskScheduler("local[*]")
- sched.backend match {
- case s: LocalSchedulerBackend =>
- assert(s.totalCores === Runtime.getRuntime.availableProcessors())
- case _ => fail()
+ val sched = createTaskScheduler("local[*]") { sched =>
+ sched.backend match {
+ case s: LocalSchedulerBackend =>
+ assert(s.totalCores === Runtime.getRuntime.availableProcessors())
+ case _ => fail()
+ }
}
}
test("local-n") {
- val sched = createTaskScheduler("local[5]")
- assert(sched.maxTaskFailures === 1)
- sched.backend match {
- case s: LocalSchedulerBackend => assert(s.totalCores === 5)
- case _ => fail()
+ val sched = createTaskScheduler("local[5]") { sched =>
+ assert(sched.maxTaskFailures === 1)
+ sched.backend match {
+ case s: LocalSchedulerBackend => assert(s.totalCores === 5)
+ case _ => fail()
+ }
}
}
test("local-*-n-failures") {
- val sched = createTaskScheduler("local[* ,2]")
- assert(sched.maxTaskFailures === 2)
- sched.backend match {
- case s: LocalSchedulerBackend =>
- assert(s.totalCores === Runtime.getRuntime.availableProcessors())
- case _ => fail()
+ val sched = createTaskScheduler("local[* ,2]") { sched =>
+ assert(sched.maxTaskFailures === 2)
+ sched.backend match {
+ case s: LocalSchedulerBackend =>
+ assert(s.totalCores === Runtime.getRuntime.availableProcessors())
+ case _ => fail()
+ }
}
}
test("local-n-failures") {
- val sched = createTaskScheduler("local[4, 2]")
- assert(sched.maxTaskFailures === 2)
- sched.backend match {
- case s: LocalSchedulerBackend => assert(s.totalCores === 4)
- case _ => fail()
+ val sched = createTaskScheduler("local[4, 2]") { sched =>
+ assert(sched.maxTaskFailures === 2)
+ sched.backend match {
+ case s: LocalSchedulerBackend => assert(s.totalCores === 4)
+ case _ => fail()
+ }
}
}
test("bad-local-n") {
val e = intercept[SparkException] {
- createTaskScheduler("local[2*]")
+ createTaskScheduler("local[2*]")()
}
assert(e.getMessage.contains("Could not parse Master URL"))
}
test("bad-local-n-failures") {
val e = intercept[SparkException] {
- createTaskScheduler("local[2*,4]")
+ createTaskScheduler("local[2*,4]")()
}
assert(e.getMessage.contains("Could not parse Master URL"))
}
test("local-default-parallelism") {
val conf = new SparkConf().set("spark.default.parallelism", "16")
- val sched = createTaskScheduler("local", "client", conf)
- sched.backend match {
- case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16)
- case _ => fail()
+ val sched = createTaskScheduler("local", "client", conf) { sched =>
+ sched.backend match {
+ case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16)
+ case _ => fail()
+ }
}
}
test("local-cluster") {
- createTaskScheduler("local-cluster[3, 14, 1024]").backend match {
- case s: StandaloneSchedulerBackend => // OK
- case _ => fail()
+ createTaskScheduler("local-cluster[3, 14, 1024]") { sched =>
+ sched.backend match {
+ case s: StandaloneSchedulerBackend => // OK
+ case _ => fail()
+ }
}
}
}
diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
index 31289026b0027..ffb679ff0dc57 100644
--- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
@@ -20,7 +20,9 @@ package org.apache.spark
// scalastyle:off
import java.io.File
-import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome}
+import scala.annotation.tailrec
+
+import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, Outcome}
import org.apache.spark.internal.Logging
import org.apache.spark.util.AccumulatorContext
@@ -52,6 +54,7 @@ import org.apache.spark.util.AccumulatorContext
abstract class SparkFunSuite
extends FunSuite
with BeforeAndAfterAll
+ with BeforeAndAfterEach
with ThreadAudit
with Logging {
// scalastyle:on
@@ -87,6 +90,47 @@ abstract class SparkFunSuite
getTestResourceFile(file).getCanonicalPath
}
+ /**
+ * Note: this method doesn't support `BeforeAndAfter`. You must use `BeforeAndAfterEach` to
+ * set up and tear down resources.
+ */
+ def testRetry(s: String, n: Int = 2)(body: => Unit): Unit = {
+ test(s) {
+ retry(n) {
+ body
+ }
+ }
+ }
+
+ /**
+ * Note: this method doesn't support `BeforeAndAfter`. You must use `BeforeAndAfterEach` to
+ * set up and tear down resources.
+ */
+ def retry[T](n: Int)(body: => T): T = {
+ if (this.isInstanceOf[BeforeAndAfter]) {
+ throw new UnsupportedOperationException(
+ s"testRetry/retry cannot be used with ${classOf[BeforeAndAfter]}. " +
+ s"Please use ${classOf[BeforeAndAfterEach]} instead.")
+ }
+ retry0(n, n)(body)
+ }
+
+ @tailrec private final def retry0[T](n: Int, n0: Int)(body: => T): T = {
+ try body
+ catch { case e: Throwable =>
+ if (n > 0) {
+ logWarning(e.getMessage, e)
+ logInfo(s"\n\n===== RETRY #${n0 - n + 1} =====\n")
+ // Reset state before re-attempting in order so that tests which use patterns like
+ // LocalSparkContext to clean up state can work correctly when retried.
+ afterEach()
+ beforeEach()
+ retry0(n-1, n0)(body)
+ }
+ else throw e
+ }
+ }
+
/**
* Log the suite name and the test name before and after each test.
*
diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
index a15ae040d43a9..75812ae02690d 100644
--- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
@@ -28,7 +28,7 @@ import org.apache.spark.JobExecutionStatus._
class StatusTrackerSuite extends SparkFunSuite with Matchers with LocalSparkContext {
- test("basic status API usage") {
+ testRetry("basic status API usage") {
sc = new SparkContext("local", "test", new SparkConf(false))
val jobFuture = sc.parallelize(1 to 10000, 2).map(identity).groupBy(identity).collectAsync()
val jobId: Int = eventually(timeout(10 seconds)) {
diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala
index 05b4e67412f2e..6f9b583898c38 100644
--- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala
@@ -18,9 +18,13 @@
package org.apache.spark.api.python
import java.io.{ByteArrayOutputStream, DataOutputStream}
+import java.net.{InetAddress, Socket}
import java.nio.charset.StandardCharsets
-import org.apache.spark.SparkFunSuite
+import scala.concurrent.duration.Duration
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.security.SocketAuthHelper
class PythonRDDSuite extends SparkFunSuite {
@@ -44,4 +48,21 @@ class PythonRDDSuite extends SparkFunSuite {
("a".getBytes(StandardCharsets.UTF_8), null),
(null, "b".getBytes(StandardCharsets.UTF_8))), buffer)
}
+
+ test("python server error handling") {
+ val authHelper = new SocketAuthHelper(new SparkConf())
+ val errorServer = new ExceptionPythonServer(authHelper)
+ val client = new Socket(InetAddress.getLoopbackAddress(), errorServer.port)
+ authHelper.authToServer(client)
+ val ex = intercept[Exception] { errorServer.getResult(Duration(1, "second")) }
+ assert(ex.getCause().getMessage().contains("exception within handleConnection"))
+ }
+
+ class ExceptionPythonServer(authHelper: SocketAuthHelper)
+ extends PythonServer[Unit](authHelper, "error-server") {
+
+ override def handleConnection(sock: Socket): Unit = {
+ throw new Exception("exception within handleConnection")
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index f829fecc30840..1b6f765b8095b 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -72,22 +72,31 @@ trait TestPrematureExit {
mainObject.printStream = printStream
@volatile var exitedCleanly = false
+ val original = mainObject.exitFn
mainObject.exitFn = (_) => exitedCleanly = true
-
- val thread = new Thread {
- override def run() = try {
- mainObject.main(input)
- } catch {
- // If exceptions occur after the "exit" has happened, fine to ignore them.
- // These represent code paths not reachable during normal execution.
- case e: Exception => if (!exitedCleanly) throw e
+ try {
+ @volatile var exception: Exception = null
+ val thread = new Thread {
+ override def run() = try {
+ mainObject.main(input)
+ } catch {
+ // Capture the exception to check whether the exception contains searchString or not
+ case e: Exception => exception = e
+ }
}
- }
- thread.start()
- thread.join()
- val joined = printStream.lineBuffer.mkString("\n")
- if (!joined.contains(searchString)) {
- fail(s"Search string '$searchString' not found in $joined")
+ thread.start()
+ thread.join()
+ if (exitedCleanly) {
+ val joined = printStream.lineBuffer.mkString("\n")
+ assert(joined.contains(searchString))
+ } else {
+ assert(exception != null)
+ if (!exception.getMessage.contains(searchString)) {
+ throw exception
+ }
+ }
+ } finally {
+ mainObject.exitFn = original
}
}
}
@@ -577,7 +586,7 @@ class SparkSubmitSuite
}
// TODO(SPARK-9603): Building a package is flaky on Jenkins Maven builds.
- // See https://gist.github.com/shivaram/3a2fecce60768a603dac for a error log
+ // See https://gist.github.com/shivaram/3a2fecce60768a603dac for an error log
ignore("correctly builds R packages included in a jar with --packages") {
assume(RUtils.isRInstalled, "R isn't installed on this machine.")
assume(RUtils.isSparkRInstalled, "SparkR is not installed in this build.")
@@ -957,6 +966,25 @@ class SparkSubmitSuite
}
}
+ test("remove copies of application jar from classpath") {
+ val fs = File.separator
+ val sparkConf = new SparkConf(false)
+ val hadoopConf = new Configuration()
+ val secMgr = new SecurityManager(sparkConf)
+
+ val appJarName = "myApp.jar"
+ val jar1Name = "myJar1.jar"
+ val jar2Name = "myJar2.jar"
+ val userJar = s"file:/path${fs}to${fs}app${fs}jar$fs$appJarName"
+ val jars = s"file:/$jar1Name,file:/$appJarName,file:/$jar2Name"
+
+ val resolvedJars = DependencyUtils
+ .resolveAndDownloadJars(jars, userJar, sparkConf, hadoopConf, secMgr)
+
+ assert(!resolvedJars.contains(appJarName))
+ assert(resolvedJars.contains(jar1Name) && resolvedJars.contains(jar2Name))
+ }
+
test("Avoid re-upload remote resources in yarn client mode") {
val hadoopConf = new Configuration()
updateConfWithFakeS3Fs(hadoopConf)
@@ -1144,6 +1172,53 @@ class SparkSubmitSuite
conf1.get(PY_FILES.key) should be (s"s3a://${pyFile.getAbsolutePath}")
conf1.get("spark.submit.pyFiles") should (startWith("/"))
}
+
+ test("handles natural line delimiters in --properties-file and --conf uniformly") {
+ val delimKey = "spark.my.delimiter."
+ val LF = "\n"
+ val CR = "\r"
+
+ val lineFeedFromCommandLine = s"${delimKey}lineFeedFromCommandLine" -> LF
+ val leadingDelimKeyFromFile = s"${delimKey}leadingDelimKeyFromFile" -> s"${LF}blah"
+ val trailingDelimKeyFromFile = s"${delimKey}trailingDelimKeyFromFile" -> s"blah${CR}"
+ val infixDelimFromFile = s"${delimKey}infixDelimFromFile" -> s"${CR}blah${LF}"
+ val nonDelimSpaceFromFile = s"${delimKey}nonDelimSpaceFromFile" -> " blah\f"
+
+ val testProps = Seq(leadingDelimKeyFromFile, trailingDelimKeyFromFile, infixDelimFromFile,
+ nonDelimSpaceFromFile)
+
+ val props = new java.util.Properties()
+ val propsFile = File.createTempFile("test-spark-conf", ".properties",
+ Utils.createTempDir())
+ val propsOutputStream = new FileOutputStream(propsFile)
+ try {
+ testProps.foreach { case (k, v) => props.put(k, v) }
+ props.store(propsOutputStream, "test whitespace")
+ } finally {
+ propsOutputStream.close()
+ }
+
+ val clArgs = Seq(
+ "--class", "org.SomeClass",
+ "--conf", s"${lineFeedFromCommandLine._1}=${lineFeedFromCommandLine._2}",
+ "--conf", "spark.master=yarn",
+ "--properties-file", propsFile.getPath,
+ "thejar.jar")
+
+ val appArgs = new SparkSubmitArguments(clArgs)
+ val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs)
+
+ Seq(
+ lineFeedFromCommandLine,
+ leadingDelimKeyFromFile,
+ trailingDelimKeyFromFile,
+ infixDelimFromFile
+ ).foreach { case (k, v) =>
+ conf.get(k) should be (v)
+ }
+
+ conf.get(nonDelimSpaceFromFile._1) should be ("blah")
+ }
}
object SparkSubmitSuite extends SparkFunSuite with TimeLimits {
diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala
index a1d2a1283db14..aa51310d50055 100644
--- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala
@@ -493,27 +493,34 @@ class StandaloneDynamicAllocationSuite
}
test("executor registration on a blacklisted host must fail") {
+ // The context isn't really used by the test, but it helps with creating a test scheduler,
+ // since CoarseGrainedSchedulerBackend makes a lot of calls to the context instance.
sc = new SparkContext(appConf.set(config.BLACKLIST_ENABLED.key, "true"))
+
val endpointRef = mock(classOf[RpcEndpointRef])
val mockAddress = mock(classOf[RpcAddress])
when(endpointRef.address).thenReturn(mockAddress)
val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty)
- // Get "localhost" on a blacklist.
val taskScheduler = mock(classOf[TaskSchedulerImpl])
when(taskScheduler.nodeBlacklist()).thenReturn(Set("blacklisted-host"))
+ when(taskScheduler.resourceOffers(any())).thenReturn(Nil)
when(taskScheduler.sc).thenReturn(sc)
- sc.taskScheduler = taskScheduler
-
- // Create a fresh scheduler backend to blacklist "localhost".
- sc.schedulerBackend.stop()
- val backend =
- new StandaloneSchedulerBackend(taskScheduler, sc, Array(masterRpcEnv.address.toSparkURL))
- backend.start()
- backend.driverEndpoint.ask[Boolean](message)
- eventually(timeout(10.seconds), interval(100.millis)) {
- verify(endpointRef).send(RegisterExecutorFailed(any()))
+ val rpcEnv = RpcEnv.create("test-rpcenv", "localhost", 0, conf, securityManager)
+ try {
+ val scheduler = new CoarseGrainedSchedulerBackend(taskScheduler, rpcEnv)
+ try {
+ scheduler.start()
+ scheduler.driverEndpoint.ask[Boolean](message)
+ eventually(timeout(10.seconds), interval(100.millis)) {
+ verify(endpointRef).send(RegisterExecutorFailed(any()))
+ }
+ } finally {
+ scheduler.stop()
+ }
+ } finally {
+ rpcEnv.shutdown()
}
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala
index a1707e6540b39..baeefea3158ef 100644
--- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark.deploy.client
+import java.io.Closeable
import java.util.concurrent.ConcurrentLinkedQueue
import scala.concurrent.duration._
@@ -85,57 +86,59 @@ class AppClientSuite
}
test("interface methods of AppClient using local Master") {
- val ci = new AppClientInst(masterRpcEnv.address.toSparkURL)
+ Utils.tryWithResource(new AppClientInst(masterRpcEnv.address.toSparkURL)) { ci =>
- ci.client.start()
+ ci.client.start()
- // Client should connect with one Master which registers the application
- eventually(timeout(10.seconds), interval(10.millis)) {
- val apps = getApplications()
- assert(ci.listener.connectedIdList.size === 1, "client listener should have one connection")
- assert(apps.size === 1, "master should have 1 registered app")
- }
+ // Client should connect with one Master which registers the application
+ eventually(timeout(10.seconds), interval(10.millis)) {
+ val apps = getApplications()
+ assert(ci.listener.connectedIdList.size === 1, "client listener should have one connection")
+ assert(apps.size === 1, "master should have 1 registered app")
+ }
- // Send message to Master to request Executors, verify request by change in executor limit
- val numExecutorsRequested = 1
- whenReady(
+ // Send message to Master to request Executors, verify request by change in executor limit
+ val numExecutorsRequested = 1
+ whenReady(
ci.client.requestTotalExecutors(numExecutorsRequested),
timeout(10.seconds),
interval(10.millis)) { acknowledged =>
- assert(acknowledged)
- }
+ assert(acknowledged)
+ }
- eventually(timeout(10.seconds), interval(10.millis)) {
- val apps = getApplications()
- assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed")
- }
+ eventually(timeout(10.seconds), interval(10.millis)) {
+ val apps = getApplications()
+ assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed")
+ }
- // Send request to kill executor, verify request was made
- val executorId: String = getApplications().head.executors.head._2.fullId
- whenReady(
+ // Send request to kill executor, verify request was made
+ val executorId: String = getApplications().head.executors.head._2.fullId
+ whenReady(
ci.client.killExecutors(Seq(executorId)),
timeout(10.seconds),
interval(10.millis)) { acknowledged =>
- assert(acknowledged)
- }
+ assert(acknowledged)
+ }
- // Issue stop command for Client to disconnect from Master
- ci.client.stop()
+ // Issue stop command for Client to disconnect from Master
+ ci.client.stop()
- // Verify Client is marked dead and unregistered from Master
- eventually(timeout(10.seconds), interval(10.millis)) {
- val apps = getApplications()
- assert(ci.listener.deadReasonList.size === 1, "client should have been marked dead")
- assert(apps.isEmpty, "master should have 0 registered apps")
+ // Verify Client is marked dead and unregistered from Master
+ eventually(timeout(10.seconds), interval(10.millis)) {
+ val apps = getApplications()
+ assert(ci.listener.deadReasonList.size === 1, "client should have been marked dead")
+ assert(apps.isEmpty, "master should have 0 registered apps")
+ }
}
}
test("request from AppClient before initialized with master") {
- val ci = new AppClientInst(masterRpcEnv.address.toSparkURL)
+ Utils.tryWithResource(new AppClientInst(masterRpcEnv.address.toSparkURL)) { ci =>
- // requests to master should fail immediately
- whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success =>
- assert(success === false)
+ // requests to master should fail immediately
+ whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success =>
+ assert(success === false)
+ }
}
}
@@ -219,13 +222,17 @@ class AppClientSuite
}
/** Create AppClient and supporting objects */
- private class AppClientInst(masterUrl: String) {
+ private class AppClientInst(masterUrl: String) extends Closeable {
val rpcEnv = RpcEnv.create("spark", Utils.localHostName(), 0, conf, securityManager)
private val cmd = new Command(TestExecutor.getClass.getCanonicalName.stripSuffix("$"),
List(), Map(), Seq(), Seq(), Seq())
private val desc = new ApplicationDescription("AppClientSuite", Some(1), 512, cmd, "ignored")
val listener = new AppClientCollector
val client = new StandaloneAppClient(rpcEnv, Array(masterUrl), desc, listener, new SparkConf)
+
+ override def close(): Unit = {
+ rpcEnv.shutdown()
+ }
}
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
index b4eba755eccbf..dc15da5ecbecf 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
@@ -27,14 +27,13 @@ import scala.concurrent.duration._
import scala.language.postfixOps
import com.google.common.io.{ByteStreams, Files}
-import org.apache.hadoop.fs.{FileStatus, Path}
-import org.apache.hadoop.hdfs.DistributedFileSystem
+import org.apache.hadoop.fs.{FileStatus, FileSystem, FSDataInputStream, Path}
+import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem}
import org.apache.hadoop.security.AccessControlException
import org.json4s.jackson.JsonMethods._
import org.mockito.ArgumentMatcher
import org.mockito.Matchers.{any, argThat}
import org.mockito.Mockito.{doThrow, mock, spy, verify, when}
-import org.scalatest.BeforeAndAfter
import org.scalatest.Matchers
import org.scalatest.concurrent.Eventually._
@@ -48,16 +47,21 @@ import org.apache.spark.status.AppStatusStore
import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo}
import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils}
-class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging {
+class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging {
private var testDir: File = null
- before {
+ override def beforeEach(): Unit = {
+ super.beforeEach()
testDir = Utils.createTempDir(namePrefix = s"a b%20c+d")
}
- after {
- Utils.deleteRecursively(testDir)
+ override def afterEach(): Unit = {
+ try {
+ Utils.deleteRecursively(testDir)
+ } finally {
+ super.afterEach()
+ }
}
/** Create a fake log file using the new log format used in Spark 1.3+ */
@@ -330,6 +334,45 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
assert(!log2.exists())
}
+ test("should not clean inprogress application with lastUpdated time less than maxTime") {
+ val firstFileModifiedTime = TimeUnit.DAYS.toMillis(1)
+ val secondFileModifiedTime = TimeUnit.DAYS.toMillis(6)
+ val maxAge = TimeUnit.DAYS.toMillis(7)
+ val clock = new ManualClock(0)
+ val provider = new FsHistoryProvider(
+ createTestConf().set(MAX_LOG_AGE_S, maxAge / 1000), clock)
+ val log = newLogFile("inProgressApp1", None, inProgress = true)
+ writeFile(log, true, None,
+ SparkListenerApplicationStart(
+ "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1"))
+ )
+ clock.setTime(firstFileModifiedTime)
+ log.setLastModified(clock.getTimeMillis())
+ provider.checkForLogs()
+ writeFile(log, true, None,
+ SparkListenerApplicationStart(
+ "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")),
+ SparkListenerJobStart(0, 1L, Nil, null)
+ )
+
+ clock.setTime(secondFileModifiedTime)
+ log.setLastModified(clock.getTimeMillis())
+ provider.checkForLogs()
+ clock.setTime(TimeUnit.DAYS.toMillis(10))
+ writeFile(log, true, None,
+ SparkListenerApplicationStart(
+ "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")),
+ SparkListenerJobStart(0, 1L, Nil, null),
+ SparkListenerJobEnd(0, 1L, JobSucceeded)
+ )
+ log.setLastModified(clock.getTimeMillis())
+ provider.checkForLogs()
+ // This should not trigger any cleanup
+ updateAndCheck(provider) { list =>
+ list.size should be(1)
+ }
+ }
+
test("log cleaner for inProgress files") {
val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10)
val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20)
@@ -448,7 +491,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
provider.inSafeMode = false
clock.setTime(10000)
- eventually(timeout(1 second), interval(10 millis)) {
+ eventually(timeout(3.second), interval(10.milliseconds)) {
provider.getConfig().keys should not contain ("HDFS State")
}
} finally {
@@ -456,7 +499,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
}
}
- test("provider reports error after FS leaves safe mode") {
+ testRetry("provider reports error after FS leaves safe mode") {
testDir.delete()
val clock = new ManualClock()
val provider = new SafeModeTestProvider(createTestConf(), clock)
@@ -466,7 +509,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
provider.inSafeMode = false
clock.setTime(10000)
- eventually(timeout(1 second), interval(10 millis)) {
+ eventually(timeout(3.second), interval(10.milliseconds)) {
verify(errorHandler).uncaughtException(any(), any())
}
} finally {
@@ -830,11 +873,12 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
writeFile(accessGranted, true, None,
SparkListenerApplicationStart("accessGranted", Some("accessGranted"), 1L, "test", None),
SparkListenerApplicationEnd(5L))
+ var isReadable = false
val mockedFs = spy(provider.fs)
doThrow(new AccessControlException("Cannot read accessDenied file")).when(mockedFs).open(
argThat(new ArgumentMatcher[Path]() {
override def matches(path: Any): Boolean = {
- path.asInstanceOf[Path].getName.toLowerCase == "accessdenied"
+ path.asInstanceOf[Path].getName.toLowerCase == "accessdenied" && !isReadable
}
}))
val mockedProvider = spy(provider)
@@ -842,9 +886,6 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
updateAndCheck(mockedProvider) { list =>
list.size should be(1)
}
- writeFile(accessDenied, true, None,
- SparkListenerApplicationStart("accessDenied", Some("accessDenied"), 1L, "test", None),
- SparkListenerApplicationEnd(5L))
// Doing 2 times in order to check the blacklist filter too
updateAndCheck(mockedProvider) { list =>
list.size should be(1)
@@ -852,8 +893,47 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
val accessDeniedPath = new Path(accessDenied.getPath)
assert(mockedProvider.isBlacklisted(accessDeniedPath))
clock.advance(24 * 60 * 60 * 1000 + 1) // add a bit more than 1d
+ isReadable = true
mockedProvider.cleanLogs()
- assert(!mockedProvider.isBlacklisted(accessDeniedPath))
+ updateAndCheck(mockedProvider) { list =>
+ assert(!mockedProvider.isBlacklisted(accessDeniedPath))
+ assert(list.exists(_.name == "accessDenied"))
+ assert(list.exists(_.name == "accessGranted"))
+ list.size should be(2)
+ }
+ }
+
+ test("check in-progress event logs absolute length") {
+ val path = new Path("testapp.inprogress")
+ val provider = new FsHistoryProvider(createTestConf())
+ val mockedProvider = spy(provider)
+ val mockedFs = mock(classOf[FileSystem])
+ val in = mock(classOf[FSDataInputStream])
+ val dfsIn = mock(classOf[DFSInputStream])
+ when(mockedProvider.fs).thenReturn(mockedFs)
+ when(mockedFs.open(path)).thenReturn(in)
+ when(in.getWrappedStream).thenReturn(dfsIn)
+ when(dfsIn.getFileLength).thenReturn(200)
+ // FileStatus.getLen is more than logInfo fileSize
+ var fileStatus = new FileStatus(200, false, 0, 0, 0, path)
+ var logInfo = new LogInfo(path.toString, 0, Some("appId"), Some("attemptId"), 100)
+ assert(mockedProvider.shouldReloadLog(logInfo, fileStatus))
+
+ fileStatus = new FileStatus()
+ fileStatus.setPath(path)
+ // DFSInputStream.getFileLength is more than logInfo fileSize
+ logInfo = new LogInfo(path.toString, 0, Some("appId"), Some("attemptId"), 100)
+ assert(mockedProvider.shouldReloadLog(logInfo, fileStatus))
+ // DFSInputStream.getFileLength is equal to logInfo fileSize
+ logInfo = new LogInfo(path.toString, 0, Some("appId"), Some("attemptId"), 200)
+ assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus))
+ // in.getWrappedStream returns other than DFSInputStream
+ val bin = mock(classOf[BufferedInputStream])
+ when(in.getWrappedStream).thenReturn(bin)
+ assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus))
+ // fs.open throws exception
+ when(mockedFs.open(path)).thenThrow(new IOException("Throwing intentionally"))
+ assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus))
}
/**
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index 11b29121739a4..4880624f9ce9d 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -634,6 +634,49 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
}
}
+ test("access history application defaults to the last attempt id") {
+
+ def getRedirectUrl(url: URL): (Int, String) = {
+ val connection = url.openConnection().asInstanceOf[HttpURLConnection]
+ connection.setRequestMethod("GET")
+ connection.setUseCaches(false)
+ connection.setDefaultUseCaches(false)
+ connection.setInstanceFollowRedirects(false)
+ connection.connect()
+ val code = connection.getResponseCode()
+ val location = connection.getHeaderField("Location")
+ (code, location)
+ }
+
+ def buildPageAttemptUrl(appId: String, attemptId: Option[Int]): URL = {
+ attemptId match {
+ case Some(id) =>
+ new URL(s"http://localhost:$port/history/$appId/$id")
+ case None =>
+ new URL(s"http://localhost:$port/history/$appId")
+ }
+ }
+
+ val oneAttemptAppId = "local-1430917381534"
+ HistoryServerSuite.getUrl(buildPageAttemptUrl(oneAttemptAppId, None))
+
+ val multiAttemptAppid = "local-1430917381535"
+ val lastAttemptId = Some(2)
+ val lastAttemptUrl = buildPageAttemptUrl(multiAttemptAppid, lastAttemptId)
+ Seq(None, Some(1), Some(2)).foreach { attemptId =>
+ val url = buildPageAttemptUrl(multiAttemptAppid, attemptId)
+ val (code, location) = getRedirectUrl(url)
+ assert(code === 302, s"Unexpected status code $code for $url")
+ attemptId match {
+ case None =>
+ assert(location.stripSuffix("/") === lastAttemptUrl.toString)
+ case _ =>
+ assert(location.stripSuffix("/") === url.toString)
+ }
+ HistoryServerSuite.getUrl(new URL(location))
+ }
+ }
+
def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = {
HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/api/v1/$path"))
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
index 84b3a29b58bf4..07830fdd3b4ce 100644
--- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
@@ -642,59 +642,70 @@ class MasterSuite extends SparkFunSuite
val masterState = master.self.askSync[MasterStateResponse](RequestMasterState)
assert(masterState.status === RecoveryState.ALIVE, "Master is not alive")
}
- val worker1 = new MockWorker(master.self)
- worker1.rpcEnv.setupEndpoint("worker", worker1)
- val worker1Reg = RegisterWorker(
- worker1.id,
- "localhost",
- 9998,
- worker1.self,
- 10,
- 1024,
- "http://localhost:8080",
- RpcAddress("localhost2", 10000))
- master.self.send(worker1Reg)
- val driver = DeployTestUtils.createDriverDesc().copy(supervise = true)
- master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver))
-
- eventually(timeout(10.seconds)) {
- assert(worker1.apps.nonEmpty)
- }
-
- eventually(timeout(10.seconds)) {
- val masterState = master.self.askSync[MasterStateResponse](RequestMasterState)
- assert(masterState.workers(0).state == WorkerState.DEAD)
- }
+ var worker1: MockWorker = null
+ var worker2: MockWorker = null
+ try {
+ worker1 = new MockWorker(master.self)
+ worker1.rpcEnv.setupEndpoint("worker", worker1)
+ val worker1Reg = RegisterWorker(
+ worker1.id,
+ "localhost",
+ 9998,
+ worker1.self,
+ 10,
+ 1024,
+ "http://localhost:8080",
+ RpcAddress("localhost2", 10000))
+ master.self.send(worker1Reg)
+ val driver = DeployTestUtils.createDriverDesc().copy(supervise = true)
+ master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver))
+
+ eventually(timeout(10.seconds)) {
+ assert(worker1.apps.nonEmpty)
+ }
- val worker2 = new MockWorker(master.self)
- worker2.rpcEnv.setupEndpoint("worker", worker2)
- master.self.send(RegisterWorker(
- worker2.id,
- "localhost",
- 9999,
- worker2.self,
- 10,
- 1024,
- "http://localhost:8081",
- RpcAddress("localhost", 10001)))
- eventually(timeout(10.seconds)) {
- assert(worker2.apps.nonEmpty)
- }
+ eventually(timeout(10.seconds)) {
+ val masterState = master.self.askSync[MasterStateResponse](RequestMasterState)
+ assert(masterState.workers(0).state == WorkerState.DEAD)
+ }
- master.self.send(worker1Reg)
- eventually(timeout(10.seconds)) {
- val masterState = master.self.askSync[MasterStateResponse](RequestMasterState)
+ worker2 = new MockWorker(master.self)
+ worker2.rpcEnv.setupEndpoint("worker", worker2)
+ master.self.send(RegisterWorker(
+ worker2.id,
+ "localhost",
+ 9999,
+ worker2.self,
+ 10,
+ 1024,
+ "http://localhost:8081",
+ RpcAddress("localhost", 10001)))
+ eventually(timeout(10.seconds)) {
+ assert(worker2.apps.nonEmpty)
+ }
- val worker = masterState.workers.filter(w => w.id == worker1.id)
- assert(worker.length == 1)
- // make sure the `DriverStateChanged` arrives at Master.
- assert(worker(0).drivers.isEmpty)
- assert(worker1.apps.isEmpty)
- assert(worker1.drivers.isEmpty)
- assert(worker2.apps.size == 1)
- assert(worker2.drivers.size == 1)
- assert(masterState.activeDrivers.length == 1)
- assert(masterState.activeApps.length == 1)
+ master.self.send(worker1Reg)
+ eventually(timeout(10.seconds)) {
+ val masterState = master.self.askSync[MasterStateResponse](RequestMasterState)
+
+ val worker = masterState.workers.filter(w => w.id == worker1.id)
+ assert(worker.length == 1)
+ // make sure the `DriverStateChanged` arrives at Master.
+ assert(worker(0).drivers.isEmpty)
+ assert(worker1.apps.isEmpty)
+ assert(worker1.drivers.isEmpty)
+ assert(worker2.apps.size == 1)
+ assert(worker2.drivers.size == 1)
+ assert(masterState.activeDrivers.length == 1)
+ assert(masterState.activeApps.length == 1)
+ }
+ } finally {
+ if (worker1 != null) {
+ worker1.rpcEnv.shutdown()
+ }
+ if (worker2 != null) {
+ worker2.rpcEnv.shutdown()
+ }
}
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala
index 54c168a8218f3..75fb716813ae7 100644
--- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala
@@ -376,6 +376,18 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach {
assert(filteredVariables == Map("SPARK_VAR" -> "1"))
}
+ test("client does not send 'SPARK_HOME' env var by default") {
+ val environmentVariables = Map("SPARK_VAR" -> "1", "SPARK_HOME" -> "1")
+ val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables)
+ assert(filteredVariables == Map("SPARK_VAR" -> "1"))
+ }
+
+ test("client does not send 'SPARK_CONF_DIR' env var by default") {
+ val environmentVariables = Map("SPARK_VAR" -> "1", "SPARK_CONF_DIR" -> "1")
+ val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables)
+ assert(filteredVariables == Map("SPARK_VAR" -> "1"))
+ }
+
test("client includes mesos env vars") {
val environmentVariables = Map("SPARK_VAR" -> "1", "MESOS_VAR" -> "1", "OTHER_VAR" -> "1")
val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables)
diff --git a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala
new file mode 100644
index 0000000000000..d7e4b9166fa04
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala
@@ -0,0 +1,104 @@
+/*
+ * 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.network
+
+import java.io.InputStream
+import java.nio.ByteBuffer
+
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+
+import org.scalatest.concurrent._
+
+import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager}
+import org.apache.spark.storage.{BlockId, StorageLevel}
+
+class BlockTransferServiceSuite extends SparkFunSuite with TimeLimits {
+
+ implicit val defaultSignaler: Signaler = ThreadSignaler
+
+ test("fetchBlockSync should not hang when BlockFetchingListener.onBlockFetchSuccess fails") {
+ // Create a mocked `BlockTransferService` to call `BlockFetchingListener.onBlockFetchSuccess`
+ // with a bad `ManagedBuffer` which will trigger an exception in `onBlockFetchSuccess`.
+ val blockTransferService = new BlockTransferService {
+ override def init(blockDataManager: BlockDataManager): Unit = {}
+
+ override def close(): Unit = {}
+
+ override def port: Int = 0
+
+ override def hostName: String = "localhost-unused"
+
+ override def fetchBlocks(
+ host: String,
+ port: Int,
+ execId: String,
+ blockIds: Array[String],
+ listener: BlockFetchingListener,
+ tempFileManager: DownloadFileManager): Unit = {
+ // Notify BlockFetchingListener with a bad ManagedBuffer asynchronously
+ new Thread() {
+ override def run(): Unit = {
+ // This is a bad buffer to trigger `IllegalArgumentException` in
+ // `BlockFetchingListener.onBlockFetchSuccess`. The real issue we hit is
+ // `ByteBuffer.allocate` throws `OutOfMemoryError`, but we cannot make it happen in
+ // a test. Instead, we use a negative size value to make `ByteBuffer.allocate` fail,
+ // and this should trigger the same code path as `OutOfMemoryError`.
+ val badBuffer = new ManagedBuffer {
+ override def size(): Long = -1
+
+ override def nioByteBuffer(): ByteBuffer = null
+
+ override def createInputStream(): InputStream = null
+
+ override def retain(): ManagedBuffer = this
+
+ override def release(): ManagedBuffer = this
+
+ override def convertToNetty(): AnyRef = null
+ }
+ listener.onBlockFetchSuccess("block-id-unused", badBuffer)
+ }
+ }.start()
+ }
+
+ override def uploadBlock(
+ hostname: String,
+ port: Int,
+ execId: String,
+ blockId: BlockId,
+ blockData: ManagedBuffer,
+ level: StorageLevel,
+ classTag: ClassTag[_]): Future[Unit] = {
+ // This method is unused in this test
+ throw new UnsupportedOperationException("uploadBlock")
+ }
+ }
+
+ val e = intercept[SparkException] {
+ failAfter(10.seconds) {
+ blockTransferService.fetchBlockSync(
+ "localhost-unused", 0, "exec-id-unused", "block-id-unused", null)
+ }
+ }
+ assert(e.getCause.isInstanceOf[IllegalArgumentException])
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
index 1a0eb250e7cdc..7d419579a36d0 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala
@@ -19,17 +19,20 @@ package org.apache.spark.rdd
import java.io.File
+import scala.collection.JavaConverters._
import scala.collection.Map
+import scala.concurrent.duration._
import scala.io.Codec
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.{LongWritable, Text}
import org.apache.hadoop.mapred.{FileSplit, JobConf, TextInputFormat}
+import org.scalatest.concurrent.Eventually
import org.apache.spark._
import org.apache.spark.util.Utils
-class PipedRDDSuite extends SparkFunSuite with SharedSparkContext {
+class PipedRDDSuite extends SparkFunSuite with SharedSparkContext with Eventually {
val envCommand = if (Utils.isWindows) {
"cmd.exe /C set"
} else {
@@ -83,6 +86,34 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext {
}
}
+ test("stdin writer thread should be exited when task is finished") {
+ assume(TestUtils.testCommandAvailable("cat"))
+ val nums = sc.makeRDD(Array(1, 2, 3, 4), 1).map { x =>
+ val obj = new Object()
+ obj.synchronized {
+ obj.wait() // make the thread waits here.
+ }
+ x
+ }
+
+ val piped = nums.pipe(Seq("cat"))
+
+ val result = piped.mapPartitions(_ => Array.emptyIntArray.iterator)
+
+ assert(result.collect().length === 0)
+
+ // SPARK-29104 PipedRDD will invoke `stdinWriterThread.interrupt()` at task completion,
+ // and `obj.wait` will get InterruptedException. However, there exists a possibility
+ // which the thread termination gets delayed because the thread starts from `obj.wait()`
+ // with that exception. To prevent test flakiness, we need to use `eventually`.
+ eventually(timeout(10.seconds), interval(1.second)) {
+ // collect stdin writer threads
+ val stdinWriterThread = Thread.getAllStackTraces.keySet().asScala
+ .find { _.getName.startsWith(PipedRDD.STDIN_WRITER_THREAD_PREFIX) }
+ assert(stdinWriterThread.isEmpty)
+ }
+ }
+
test("advanced pipe") {
assume(TestUtils.testCommandAvailable("cat"))
val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala
index f9481f875d439..59b4b706bbcdd 100644
--- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala
@@ -17,13 +17,20 @@
package org.apache.spark.rpc.netty
+import java.util.concurrent.ExecutionException
+
+import scala.concurrent.duration._
+
+import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits}
import org.scalatest.mockito.MockitoSugar
import org.apache.spark._
import org.apache.spark.network.client.TransportClient
import org.apache.spark.rpc._
-class NettyRpcEnvSuite extends RpcEnvSuite with MockitoSugar {
+class NettyRpcEnvSuite extends RpcEnvSuite with MockitoSugar with TimeLimits {
+
+ private implicit val signaler: Signaler = ThreadSignaler
override def createRpcEnv(
conf: SparkConf,
@@ -84,4 +91,48 @@ class NettyRpcEnvSuite extends RpcEnvSuite with MockitoSugar {
msg3,
RequestMessage(nettyEnv, client, msg3.serialize(nettyEnv)))
}
+
+ test("StackOverflowError should be sent back and Dispatcher should survive") {
+ val numUsableCores = 2
+ val conf = new SparkConf
+ val config = RpcEnvConfig(
+ conf,
+ "test",
+ "localhost",
+ "localhost",
+ 0,
+ new SecurityManager(conf),
+ numUsableCores,
+ clientMode = false)
+ val anotherEnv = new NettyRpcEnvFactory().create(config)
+ anotherEnv.setupEndpoint("StackOverflowError", new RpcEndpoint {
+ override val rpcEnv = anotherEnv
+
+ override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
+ // scalastyle:off throwerror
+ case msg: String => throw new StackOverflowError
+ // scalastyle:on throwerror
+ case num: Int => context.reply(num)
+ }
+ })
+
+ val rpcEndpointRef = env.setupEndpointRef(anotherEnv.address, "StackOverflowError")
+ try {
+ // Send `numUsableCores` messages to trigger `numUsableCores` `StackOverflowError`s
+ for (_ <- 0 until numUsableCores) {
+ val e = intercept[SparkException] {
+ rpcEndpointRef.askSync[String]("hello")
+ }
+ // The root cause `e.getCause.getCause` because it is boxed by Scala Promise.
+ assert(e.getCause.isInstanceOf[ExecutionException])
+ assert(e.getCause.getCause.isInstanceOf[StackOverflowError])
+ }
+ failAfter(10.seconds) {
+ assert(rpcEndpointRef.askSync[Int](100) === 100)
+ }
+ } finally {
+ anotherEnv.shutdown()
+ anotherEnv.awaitTermination()
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala
index d3bbfd11d406d..29bb8232f44f5 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala
@@ -24,7 +24,6 @@ import org.apache.spark.internal.config
class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{
val badHost = "host-0"
- val duration = Duration(10, SECONDS)
/**
* This backend just always fails if the task is executed on a bad host, but otherwise succeeds
@@ -97,15 +96,16 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM
assertDataStructuresEmpty(noFailure = true)
}
- // Make sure that if we've failed on all executors, but haven't hit task.maxFailures yet, the job
- // doesn't hang
+ // Make sure that if we've failed on all executors, but haven't hit task.maxFailures yet, we try
+ // to acquire a new executor and if we aren't able to get one, the job doesn't hang and we abort
testScheduler(
"SPARK-15865 Progress with fewer executors than maxTaskFailures",
extraConfs = Seq(
config.BLACKLIST_ENABLED.key -> "true",
"spark.testing.nHosts" -> "2",
"spark.testing.nExecutorsPerHost" -> "1",
- "spark.testing.nCoresPerExecutor" -> "1"
+ "spark.testing.nCoresPerExecutor" -> "1",
+ "spark.scheduler.blacklist.unschedulableTaskSetTimeout" -> "0s"
)
) {
def runBackend(): Unit = {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala
index 80c9c6f0422a8..c5a39669366ce 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala
@@ -30,6 +30,8 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer}
class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext
with Eventually {
+ private val executorUpTimeout = 60.seconds
+
test("serialized task larger than max RPC message size") {
val conf = new SparkConf
conf.set("spark.rpc.message.maxSize", "1")
@@ -51,7 +53,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo
.setMaster("local-cluster[4, 3, 1024]")
.setAppName("test")
sc = new SparkContext(conf)
- eventually(timeout(10.seconds)) {
+ eventually(timeout(executorUpTimeout)) {
// Ensure all executors have been launched.
assert(sc.getExecutorIds().length == 4)
}
@@ -64,7 +66,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo
.setMaster("local-cluster[4, 3, 1024]")
.setAppName("test")
sc = new SparkContext(conf)
- eventually(timeout(10.seconds)) {
+ eventually(timeout(executorUpTimeout)) {
// Ensure all executors have been launched.
assert(sc.getExecutorIds().length == 4)
}
@@ -96,7 +98,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo
try {
sc.addSparkListener(listener)
- eventually(timeout(10.seconds)) {
+ eventually(timeout(executorUpTimeout)) {
// Ensure all executors have been launched.
assert(sc.getExecutorIds().length == 4)
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 4e87deb136df6..773232947b6db 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -162,32 +162,67 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
/** Length of time to wait while draining listener events. */
val WAIT_TIMEOUT_MILLIS = 10000
- val sparkListener = new SparkListener() {
- val submittedStageInfos = new HashSet[StageInfo]
- val successfulStages = new HashSet[Int]
- val failedStages = new ArrayBuffer[Int]
- val stageByOrderOfExecution = new ArrayBuffer[Int]
- val endedTasks = new HashSet[Long]
+
+ /**
+ * Listeners which records some information to verify in UTs. Getter-kind methods in this class
+ * ensures the value is returned after ensuring there's no event to process, as well as the
+ * value is immutable: prevent showing odd result by race condition.
+ */
+ class EventInfoRecordingListener extends SparkListener {
+ private val _submittedStageInfos = new HashSet[StageInfo]
+ private val _successfulStages = new HashSet[Int]
+ private val _failedStages = new ArrayBuffer[Int]
+ private val _stageByOrderOfExecution = new ArrayBuffer[Int]
+ private val _endedTasks = new HashSet[Long]
override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) {
- submittedStageInfos += stageSubmitted.stageInfo
+ _submittedStageInfos += stageSubmitted.stageInfo
}
override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) {
val stageInfo = stageCompleted.stageInfo
- stageByOrderOfExecution += stageInfo.stageId
+ _stageByOrderOfExecution += stageInfo.stageId
if (stageInfo.failureReason.isEmpty) {
- successfulStages += stageInfo.stageId
+ _successfulStages += stageInfo.stageId
} else {
- failedStages += stageInfo.stageId
+ _failedStages += stageInfo.stageId
}
}
override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
- endedTasks += taskEnd.taskInfo.taskId
+ _endedTasks += taskEnd.taskInfo.taskId
+ }
+
+ def submittedStageInfos: Set[StageInfo] = {
+ waitForListeners()
+ _submittedStageInfos.toSet
+ }
+
+ def successfulStages: Set[Int] = {
+ waitForListeners()
+ _successfulStages.toSet
+ }
+
+ def failedStages: List[Int] = {
+ waitForListeners()
+ _failedStages.toList
}
+
+ def stageByOrderOfExecution: List[Int] = {
+ waitForListeners()
+ _stageByOrderOfExecution.toList
+ }
+
+ def endedTasks: Set[Long] = {
+ waitForListeners()
+ _endedTasks.toSet
+ }
+
+ private def waitForListeners(): Unit = sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
}
+ var sparkListener: EventInfoRecordingListener = null
+
var mapOutputTracker: MapOutputTrackerMaster = null
var broadcastManager: BroadcastManager = null
var securityMgr: SecurityManager = null
@@ -236,10 +271,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
private def init(testConf: SparkConf): Unit = {
sc = new SparkContext("local[2]", "DAGSchedulerSuite", testConf)
- sparkListener.submittedStageInfos.clear()
- sparkListener.successfulStages.clear()
- sparkListener.failedStages.clear()
- sparkListener.endedTasks.clear()
+ sparkListener = new EventInfoRecordingListener
failure = null
sc.addSparkListener(sparkListener)
taskSets.clear()
@@ -361,11 +393,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
}
test("[SPARK-3353] parent stage should have lower stage id") {
- sparkListener.stageByOrderOfExecution.clear()
sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _, 4).count()
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
- assert(sparkListener.stageByOrderOfExecution.length === 2)
- assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1))
+ val stageByOrderOfExecution = sparkListener.stageByOrderOfExecution
+ assert(stageByOrderOfExecution.length === 2)
+ assert(stageByOrderOfExecution(0) < stageByOrderOfExecution(1))
}
/**
@@ -443,17 +474,17 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
// map stage1 completes successfully, with one task on each executor
complete(taskSets(0), Seq(
(Success,
- MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 1)),
+ MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))),
(Success,
- MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 1)),
+ MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))),
(Success, makeMapStatus("hostB", 1))
))
// map stage2 completes successfully, with one task on each executor
complete(taskSets(1), Seq(
(Success,
- MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 1)),
+ MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))),
(Success,
- MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 1)),
+ MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))),
(Success, makeMapStatus("hostB", 1))
))
// make sure our test setup is correct
@@ -606,9 +637,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
submit(unserializableRdd, Array(0))
assert(failure.getMessage.startsWith(
"Job aborted due to stage failure: Task not serializable:"))
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
- assert(sparkListener.failedStages.contains(0))
- assert(sparkListener.failedStages.size === 1)
+ assert(sparkListener.failedStages === Seq(0))
assertDataStructuresEmpty()
}
@@ -616,9 +645,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
submit(new MyRDD(sc, 1, Nil), Array(0))
failed(taskSets(0), "some failure")
assert(failure.getMessage === "Job aborted due to stage failure: some failure")
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
- assert(sparkListener.failedStages.contains(0))
- assert(sparkListener.failedStages.size === 1)
+ assert(sparkListener.failedStages === Seq(0))
assertDataStructuresEmpty()
}
@@ -627,9 +654,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
val jobId = submit(rdd, Array(0))
cancel(jobId)
assert(failure.getMessage === s"Job $jobId cancelled ")
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
- assert(sparkListener.failedStages.contains(0))
- assert(sparkListener.failedStages.size === 1)
+ assert(sparkListener.failedStages === Seq(0))
assertDataStructuresEmpty()
}
@@ -683,7 +708,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
assert(results === Map(0 -> 42))
assertDataStructuresEmpty()
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(sparkListener.failedStages.isEmpty)
assert(sparkListener.successfulStages.contains(0))
}
@@ -1068,7 +1092,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
taskSets(1).tasks(0),
FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"),
null))
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(sparkListener.failedStages.contains(1))
// The second ResultTask fails, with a fetch failure for the output from the second mapper.
@@ -1077,8 +1100,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"),
null))
// The SparkListener should not receive redundant failure events.
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
- assert(sparkListener.failedStages.size == 1)
+ assert(sparkListener.failedStages.size === 1)
}
test("Retry all the tasks on a resubmitted attempt of a barrier stage caused by FetchFailure") {
@@ -1183,7 +1205,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
}
// The map stage should have been submitted.
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(countSubmittedMapStageAttempts() === 1)
complete(taskSets(0), Seq(
@@ -1200,12 +1221,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
taskSets(1).tasks(0),
FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"),
null))
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(sparkListener.failedStages.contains(1))
// Trigger resubmission of the failed map stage.
runEvent(ResubmitFailedStages)
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
// Another attempt for the map stage should have been submitted, resulting in 2 total attempts.
assert(countSubmittedMapStageAttempts() === 2)
@@ -1222,7 +1241,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
// shouldn't effect anything -- our calling it just makes *SURE* it gets called between the
// desired event and our check.
runEvent(ResubmitFailedStages)
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(countSubmittedMapStageAttempts() === 2)
}
@@ -1247,7 +1265,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
}
// The map stage should have been submitted.
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(countSubmittedMapStageAttempts() === 1)
// Complete the map stage.
@@ -1256,7 +1273,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
(Success, makeMapStatus("hostB", 2))))
// The reduce stage should have been submitted.
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(countSubmittedReduceStageAttempts() === 1)
// The first result task fails, with a fetch failure for the output from the first mapper.
@@ -1271,7 +1287,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
// Because the map stage finished, another attempt for the reduce stage should have been
// submitted, resulting in 2 total attempts for each the map and the reduce stage.
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(countSubmittedMapStageAttempts() === 2)
assert(countSubmittedReduceStageAttempts() === 2)
@@ -1301,11 +1316,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
runEvent(makeCompletionEvent(
taskSets(0).tasks(1), Success, 42,
Seq.empty, createFakeTaskInfoWithId(1)))
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
// verify stage exists
assert(scheduler.stageIdToStage.contains(0))
- assert(sparkListener.endedTasks.size == 2)
-
+ assert(sparkListener.endedTasks.size === 2)
// finish other 2 tasks
runEvent(makeCompletionEvent(
taskSets(0).tasks(2), Success, 42,
@@ -1313,8 +1326,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
runEvent(makeCompletionEvent(
taskSets(0).tasks(3), Success, 42,
Seq.empty, createFakeTaskInfoWithId(3)))
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
- assert(sparkListener.endedTasks.size == 4)
+ assert(sparkListener.endedTasks.size === 4)
// verify the stage is done
assert(!scheduler.stageIdToStage.contains(0))
@@ -1324,14 +1336,12 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
runEvent(makeCompletionEvent(
taskSets(0).tasks(3), Success, 42,
Seq.empty, createFakeTaskInfoWithId(5)))
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(sparkListener.endedTasks.size == 5)
// make sure non successful tasks also send out event
runEvent(makeCompletionEvent(
taskSets(0).tasks(3), UnknownReason, 42,
Seq.empty, createFakeTaskInfoWithId(6)))
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(sparkListener.endedTasks.size == 6)
}
@@ -1405,7 +1415,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
// Listener bus should get told about the map stage failing, but not the reduce stage
// (since the reduce stage hasn't been started yet).
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(sparkListener.failedStages.toSet === Set(0))
assertDataStructuresEmpty()
@@ -1649,7 +1658,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
assert(cancelledStages.toSet === Set(0, 2))
// Make sure the listeners got told about both failed stages.
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(sparkListener.successfulStages.isEmpty)
assert(sparkListener.failedStages.toSet === Set(0, 2))
@@ -1877,6 +1885,26 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
assert(sc.parallelize(1 to 10, 2).count() === 10)
}
+ test("misbehaved accumulator should not impact other accumulators") {
+ val bad = new LongAccumulator {
+ override def merge(other: AccumulatorV2[java.lang.Long, java.lang.Long]): Unit = {
+ throw new DAGSchedulerSuiteDummyException
+ }
+ }
+ sc.register(bad, "bad")
+ val good = sc.longAccumulator("good")
+
+ sc.parallelize(1 to 10, 2).foreach { item =>
+ bad.add(1)
+ good.add(1)
+ }
+
+ // This is to ensure the `bad` accumulator did fail to update its value
+ assert(bad.value == 0L)
+ // Should be able to update the "good" accumulator
+ assert(good.value == 10L)
+ }
+
/**
* The job will be failed on first task throwing a DAGSchedulerSuiteDummyException.
* Any subsequent task WILL throw a legitimate java.lang.UnsupportedOperationException.
@@ -2587,7 +2615,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
}
// The map stage should have been submitted.
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(countSubmittedMapStageAttempts() === 1)
// The first map task fails with TaskKilled.
@@ -2605,7 +2632,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
// Trigger resubmission of the failed map stage.
runEvent(ResubmitFailedStages)
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
// Another attempt for the map stage should have been submitted, resulting in 2 total attempts.
assert(countSubmittedMapStageAttempts() === 2)
@@ -2624,7 +2650,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
}
// The map stage should have been submitted.
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(countSubmittedMapStageAttempts() === 1)
// The first map task fails with TaskKilled.
@@ -2636,7 +2661,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
// Trigger resubmission of the failed map stage.
runEvent(ResubmitFailedStages)
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
// Another attempt for the map stage should have been submitted, resulting in 2 total attempts.
assert(countSubmittedMapStageAttempts() === 2)
@@ -2649,7 +2673,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
// The second map task failure doesn't trigger stage retry.
runEvent(ResubmitFailedStages)
- sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)
assert(countSubmittedMapStageAttempts() === 2)
}
@@ -2684,27 +2707,22 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"),
null))
- val failedStages = scheduler.failedStages.toSeq
- assert(failedStages.length == 2)
- // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry.
- assert(failedStages.collect {
- case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage
- }.head.findMissingPartitions() == Seq(0))
- // The result stage is still waiting for its 2 tasks to complete
- assert(failedStages.collect {
- case stage: ResultStage => stage
- }.head.findMissingPartitions() == Seq(0, 1))
+ // The second shuffle map stage need to rerun, the job will abort for the indeterminate
+ // stage rerun.
+ assert(failure != null && failure.getMessage
+ .contains("Spark cannot rollback the ShuffleMapStage 1"))
+ }
- scheduler.resubmitFailedStages()
+ test("SPARK-29042: Sampled RDD with unordered input should be indeterminate") {
+ val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = false)
- // The first task of the `shuffleMapRdd2` failed with fetch failure
- runEvent(makeCompletionEvent(
- taskSets(3).tasks(0),
- FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"),
- null))
+ val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2))
+ val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker)
- // The job should fail because Spark can't rollback the shuffle map stage.
- assert(failure != null && failure.getMessage.contains("Spark cannot rollback"))
+ assert(shuffleMapRdd2.outputDeterministicLevel == DeterministicLevel.UNORDERED)
+
+ val sampledRdd = shuffleMapRdd2.sample(true, 0.3, 1000L)
+ assert(sampledRdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE)
}
private def assertResultStageFailToRollback(mapRdd: MyRDD): Unit = {
@@ -2799,6 +2817,33 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
assertResultStageFailToRollback(shuffleMapRdd)
}
+ test("SPARK-28699: abort stage if parent stage is indeterminate stage") {
+ val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true)
+
+ val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
+ val shuffleId = shuffleDep.shuffleId
+ val finalRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker)
+
+ submit(finalRdd, Array(0, 1))
+
+ // Finish the first shuffle map stage.
+ complete(taskSets(0), Seq(
+ (Success, makeMapStatus("hostA", 2)),
+ (Success, makeMapStatus("hostB", 2))))
+ assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty))
+
+ runEvent(makeCompletionEvent(
+ taskSets(1).tasks(0),
+ FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"),
+ null))
+
+ // Shuffle blocks of "hostA" is lost, so first task of the `shuffleMapRdd` needs to retry.
+ // The result stage is still waiting for its 2 tasks to complete.
+ // Because of shuffleMapRdd is indeterminate, this job will be abort.
+ assert(failure != null && failure.getMessage
+ .contains("Spark cannot rollback the ShuffleMapStage 0"))
+ }
+
/**
* Assert that the supplied TaskSet has exactly the given hosts as its preferred locations.
* Note that this checks only the host and not the executor ID.
@@ -2854,7 +2899,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
object DAGSchedulerSuite {
def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2): MapStatus =
- MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes), 1)
+ MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes))
def makeBlockManagerId(host: String): BlockManagerId =
BlockManagerId("exec-" + host, host, 12345)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
index b29d32f7b35c5..abc8841ac03da 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
@@ -42,15 +42,23 @@ object FakeTask {
* locations for each task (given as varargs) if this sequence is not empty.
*/
def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = {
- createTaskSet(numTasks, stageAttemptId = 0, prefLocs: _*)
+ createTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, prefLocs: _*)
}
- def createTaskSet(numTasks: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = {
- createTaskSet(numTasks, stageId = 0, stageAttemptId, prefLocs: _*)
+ def createTaskSet(
+ numTasks: Int,
+ stageId: Int,
+ stageAttemptId: Int,
+ prefLocs: Seq[TaskLocation]*): TaskSet = {
+ createTaskSet(numTasks, stageId, stageAttemptId, priority = 0, prefLocs: _*)
}
- def createTaskSet(numTasks: Int, stageId: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*):
- TaskSet = {
+ def createTaskSet(
+ numTasks: Int,
+ stageId: Int,
+ stageAttemptId: Int,
+ priority: Int,
+ prefLocs: Seq[TaskLocation]*): TaskSet = {
if (prefLocs.size != 0 && prefLocs.size != numTasks) {
throw new IllegalArgumentException("Wrong number of task locations")
}
@@ -65,6 +73,15 @@ object FakeTask {
stageId: Int,
stageAttemptId: Int,
prefLocs: Seq[TaskLocation]*): TaskSet = {
+ createShuffleMapTaskSet(numTasks, stageId, stageAttemptId, priority = 0, prefLocs: _*)
+ }
+
+ def createShuffleMapTaskSet(
+ numTasks: Int,
+ stageId: Int,
+ stageAttemptId: Int,
+ priority: Int,
+ prefLocs: Seq[TaskLocation]*): TaskSet = {
if (prefLocs.size != 0 && prefLocs.size != numTasks) {
throw new IllegalArgumentException("Wrong number of task locations")
}
@@ -74,17 +91,18 @@ object FakeTask {
}, prefLocs(i), new Properties,
SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array())
}
- new TaskSet(tasks, stageId, stageAttemptId, priority = 0, null)
+ new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null)
}
def createBarrierTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = {
- createBarrierTaskSet(numTasks, stageId = 0, stageAttempId = 0, prefLocs: _*)
+ createBarrierTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, prefLocs: _*)
}
def createBarrierTaskSet(
numTasks: Int,
stageId: Int,
- stageAttempId: Int,
+ stageAttemptId: Int,
+ priority: Int,
prefLocs: Seq[TaskLocation]*): TaskSet = {
if (prefLocs.size != 0 && prefLocs.size != numTasks) {
throw new IllegalArgumentException("Wrong number of task locations")
@@ -92,6 +110,6 @@ object FakeTask {
val tasks = Array.tabulate[Task[_]](numTasks) { i =>
new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil, isBarrier = true)
}
- new TaskSet(tasks, stageId, stageAttempId, priority = 0, null)
+ new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null)
}
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala
index 555e48bd28aa0..2155a0f2b6c21 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala
@@ -60,7 +60,7 @@ class MapStatusSuite extends SparkFunSuite {
stddev <- Seq(0.0, 0.01, 0.5, 1.0)
) {
val sizes = Array.fill[Long](numSizes)(abs(round(Random.nextGaussian() * stddev)) + mean)
- val status = MapStatus(BlockManagerId("a", "b", 10), sizes, 1)
+ val status = MapStatus(BlockManagerId("a", "b", 10), sizes)
val status1 = compressAndDecompressMapStatus(status)
for (i <- 0 until numSizes) {
if (sizes(i) != 0) {
@@ -74,7 +74,7 @@ class MapStatusSuite extends SparkFunSuite {
test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) {
val sizes = Array.fill[Long](2001)(150L)
- val status = MapStatus(null, sizes, 1)
+ val status = MapStatus(null, sizes)
assert(status.isInstanceOf[HighlyCompressedMapStatus])
assert(status.getSizeForBlock(10) === 150L)
assert(status.getSizeForBlock(50) === 150L)
@@ -86,7 +86,7 @@ class MapStatusSuite extends SparkFunSuite {
val sizes = Array.tabulate[Long](3000) { i => i.toLong }
val avg = sizes.sum / sizes.count(_ != 0)
val loc = BlockManagerId("a", "b", 10)
- val status = MapStatus(loc, sizes, 1)
+ val status = MapStatus(loc, sizes)
val status1 = compressAndDecompressMapStatus(status)
assert(status1.isInstanceOf[HighlyCompressedMapStatus])
assert(status1.location == loc)
@@ -108,7 +108,7 @@ class MapStatusSuite extends SparkFunSuite {
val smallBlockSizes = sizes.filter(n => n > 0 && n < threshold)
val avg = smallBlockSizes.sum / smallBlockSizes.length
val loc = BlockManagerId("a", "b", 10)
- val status = MapStatus(loc, sizes, 1)
+ val status = MapStatus(loc, sizes)
val status1 = compressAndDecompressMapStatus(status)
assert(status1.isInstanceOf[HighlyCompressedMapStatus])
assert(status1.location == loc)
@@ -164,7 +164,7 @@ class MapStatusSuite extends SparkFunSuite {
SparkEnv.set(env)
// Value of element in sizes is equal to the corresponding index.
val sizes = (0L to 2000L).toArray
- val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes, 1)
+ val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes)
val arrayStream = new ByteArrayOutputStream(102400)
val objectOutputStream = new ObjectOutputStream(arrayStream)
assert(status1.isInstanceOf[HighlyCompressedMapStatus])
@@ -188,32 +188,4 @@ class MapStatusSuite extends SparkFunSuite {
assert(count === 3000)
}
}
-
- test("SPARK-24519: HighlyCompressedMapStatus has configurable threshold") {
- val conf = new SparkConf()
- val env = mock(classOf[SparkEnv])
- doReturn(conf).when(env).conf
- SparkEnv.set(env)
- val sizes = Array.fill[Long](500)(150L)
- // Test default value
- val status = MapStatus(null, sizes, 1)
- assert(status.isInstanceOf[CompressedMapStatus])
- // Test Non-positive values
- for (s <- -1 to 0) {
- assertThrows[IllegalArgumentException] {
- conf.set(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS, s)
- val status = MapStatus(null, sizes, 1)
- }
- }
- // Test positive values
- Seq(1, 100, 499, 500, 501).foreach { s =>
- conf.set(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS, s)
- val status = MapStatus(null, sizes, 1)
- if(sizes.length > s) {
- assert(status.isInstanceOf[HighlyCompressedMapStatus])
- } else {
- assert(status.isInstanceOf[CompressedMapStatus])
- }
- }
- }
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
index 2d409d94ca1b3..ff0f99b5c94d0 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala
@@ -51,6 +51,9 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa
var taskScheduler: TestTaskScheduler = null
var scheduler: DAGScheduler = null
var backend: T = _
+ // Even though the tests aren't doing much, occassionally we see flakiness from pauses over
+ // a second (probably from GC?) so we leave a long timeout in here
+ val duration = Duration(10, SECONDS)
override def beforeEach(): Unit = {
if (taskScheduler != null) {
@@ -539,7 +542,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor
}
withBackend(runBackend _) {
val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray)
- val duration = Duration(1, SECONDS)
awaitJobTermination(jobFuture, duration)
}
assert(results === (0 until 10).map { _ -> 42 }.toMap)
@@ -592,7 +594,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor
}
withBackend(runBackend _) {
val jobFuture = submit(d, (0 until 30).toArray)
- val duration = Duration(1, SECONDS)
awaitJobTermination(jobFuture, duration)
}
assert(results === (0 until 30).map { idx => idx -> (4321 + idx) }.toMap)
@@ -634,7 +635,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor
}
withBackend(runBackend _) {
val jobFuture = submit(shuffledRdd, (0 until 10).toArray)
- val duration = Duration(1, SECONDS)
awaitJobTermination(jobFuture, duration)
}
assertDataStructuresEmpty()
@@ -649,7 +649,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor
}
withBackend(runBackend _) {
val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray)
- val duration = Duration(1, SECONDS)
awaitJobTermination(jobFuture, duration)
assert(failure.getMessage.contains("test task failure"))
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index 6ffd1e84f7adb..0b843be390699 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -531,6 +531,47 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
}
}
+ Seq(true, false).foreach { throwInterruptedException =>
+ val suffix = if (throwInterruptedException) "throw interrupt" else "set Thread interrupted"
+ test(s"SPARK-30285: Fix deadlock in AsyncEventQueue.removeListenerOnError: $suffix") {
+ val LISTENER_BUS_STOP_WAITING_TIMEOUT_MILLIS = 10 * 1000L // 10 seconds
+ val bus = new LiveListenerBus(new SparkConf(false))
+ val counter1 = new BasicJobCounter()
+ val counter2 = new BasicJobCounter()
+ val interruptingListener = new DelayInterruptingJobCounter(throwInterruptedException, 3)
+ bus.addToSharedQueue(counter1)
+ bus.addToSharedQueue(interruptingListener)
+ bus.addToEventLogQueue(counter2)
+ assert(bus.activeQueues() === Set(SHARED_QUEUE, EVENT_LOG_QUEUE))
+ assert(bus.findListenersByClass[BasicJobCounter]().size === 2)
+ assert(bus.findListenersByClass[DelayInterruptingJobCounter]().size === 1)
+
+ bus.start(mockSparkContext, mockMetricsSystem)
+
+ (0 until 5).foreach { jobId =>
+ bus.post(SparkListenerJobEnd(jobId, jobCompletionTime, JobSucceeded))
+ }
+
+ // Call bus.stop in a separate thread, otherwise we will block here until bus is stopped
+ val stoppingThread = new Thread(new Runnable() {
+ override def run(): Unit = bus.stop()
+ })
+ stoppingThread.start()
+ // Notify interrupting listener starts to work
+ interruptingListener.sleep = false
+ // Wait for bus to stop
+ stoppingThread.join(LISTENER_BUS_STOP_WAITING_TIMEOUT_MILLIS)
+
+ // Stopping has been finished
+ assert(stoppingThread.isAlive === false)
+ // All queues are removed
+ assert(bus.activeQueues() === Set.empty)
+ assert(counter1.count === 5)
+ assert(counter2.count === 5)
+ assert(interruptingListener.count === 3)
+ }
+ }
+
/**
* Assert that the given list of numbers has an average that is greater than zero.
*/
@@ -601,6 +642,35 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
}
}
}
+
+ /**
+ * A simple listener that works as follows:
+ * 1. sleep and wait when `sleep` is true
+ * 2. when `sleep` is false, start to work:
+ * if it is interruptOnJobId, interrupt
+ * else count SparkListenerJobEnd numbers
+ */
+ private class DelayInterruptingJobCounter(
+ val throwInterruptedException: Boolean,
+ val interruptOnJobId: Int) extends SparkListener {
+ @volatile var sleep = true
+ var count = 0
+
+ override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
+ while (sleep) {
+ Thread.sleep(10)
+ }
+ if (interruptOnJobId == jobEnd.jobId) {
+ if (throwInterruptedException) {
+ throw new InterruptedException("got interrupted")
+ } else {
+ Thread.currentThread().interrupt()
+ }
+ } else {
+ count += 1
+ }
+ }
+ }
}
// These classes can't be declared inside of the SparkListenerSuite class because we don't want
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala
index 123f7f49d21b5..a6576e0d1c520 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala
@@ -19,25 +19,23 @@ package org.apache.spark.scheduler
import scala.collection.mutable
-import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
-
import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite, TestUtils}
import org.apache.spark.scheduler.cluster.ExecutorInfo
/**
* Unit tests for SparkListener that require a local cluster.
*/
-class SparkListenerWithClusterSuite extends SparkFunSuite with LocalSparkContext
- with BeforeAndAfter with BeforeAndAfterAll {
+class SparkListenerWithClusterSuite extends SparkFunSuite with LocalSparkContext {
/** Length of time to wait while draining listener events. */
val WAIT_TIMEOUT_MILLIS = 10000
- before {
+ override def beforeEach(): Unit = {
+ super.beforeEach()
sc = new SparkContext("local-cluster[2,1,1024]", "SparkListenerSuite")
}
- test("SparkListener sends executor added message") {
+ testRetry("SparkListener sends executor added message") {
val listener = new SaveExecutorInfo
sc.addSparkListener(listener)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
index 1bddba8f6c82b..e32d38092201a 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
@@ -34,6 +34,7 @@ import org.scalatest.BeforeAndAfter
import org.scalatest.concurrent.Eventually._
import org.apache.spark._
+import org.apache.spark.TaskState.TaskState
import org.apache.spark.TestUtils.JavaSourceFromString
import org.apache.spark.storage.TaskResultBlockId
import org.apache.spark.util.{MutableURLClassLoader, RpcUtils, Utils}
@@ -78,6 +79,16 @@ private class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: Task
}
}
+private class DummyTaskSchedulerImpl(sc: SparkContext)
+ extends TaskSchedulerImpl(sc, 1, true) {
+ override def handleFailedTask(
+ taskSetManager: TaskSetManager,
+ tid: Long,
+ taskState: TaskState,
+ reason: TaskFailedReason): Unit = {
+ // do nothing
+ }
+}
/**
* A [[TaskResultGetter]] that stores the [[DirectTaskResult]]s it receives from executors
@@ -130,6 +141,31 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local
"Expect result to be removed from the block manager.")
}
+ test("handling total size of results larger than maxResultSize") {
+ sc = new SparkContext("local", "test", conf)
+ val scheduler = new DummyTaskSchedulerImpl(sc)
+ val spyScheduler = spy(scheduler)
+ val resultGetter = new TaskResultGetter(sc.env, spyScheduler)
+ scheduler.taskResultGetter = resultGetter
+ val myTsm = new TaskSetManager(spyScheduler, FakeTask.createTaskSet(2), 1) {
+ // always returns false
+ override def canFetchMoreResults(size: Long): Boolean = false
+ }
+ val indirectTaskResult = IndirectTaskResult(TaskResultBlockId(0), 0)
+ val directTaskResult = new DirectTaskResult(ByteBuffer.allocate(0), Nil)
+ val ser = sc.env.closureSerializer.newInstance()
+ val serializedIndirect = ser.serialize(indirectTaskResult)
+ val serializedDirect = ser.serialize(directTaskResult)
+ resultGetter.enqueueSuccessfulTask(myTsm, 0, serializedDirect)
+ resultGetter.enqueueSuccessfulTask(myTsm, 1, serializedIndirect)
+ eventually(timeout(1.second)) {
+ verify(spyScheduler, times(1)).handleFailedTask(
+ myTsm, 0, TaskState.KILLED, TaskKilled("Tasks result size has exceeded maxResultSize"))
+ verify(spyScheduler, times(1)).handleFailedTask(
+ myTsm, 1, TaskState.KILLED, TaskKilled("Tasks result size has exceeded maxResultSize"))
+ }
+ }
+
test("task retried if result missing from block manager") {
// Set the maximum number of task failures to > 0, so that the task set isn't aborted
// after the result is missing.
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
index 9e1d13e369ad9..ecbb6ab519dcf 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
@@ -20,10 +20,12 @@ package org.apache.spark.scheduler
import java.nio.ByteBuffer
import scala.collection.mutable.HashMap
+import scala.concurrent.duration._
import org.mockito.Matchers.{anyInt, anyObject, anyString, eq => meq}
import org.mockito.Mockito.{atLeast, atMost, never, spy, times, verify, when}
import org.scalatest.BeforeAndAfterEach
+import org.scalatest.concurrent.Eventually
import org.scalatest.mockito.MockitoSugar
import org.apache.spark._
@@ -40,7 +42,7 @@ class FakeSchedulerBackend extends SchedulerBackend {
}
class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach
- with Logging with MockitoSugar {
+ with Logging with MockitoSugar with Eventually {
var failedTaskSetException: Option[Throwable] = None
var failedTaskSetReason: String = null
@@ -75,17 +77,23 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
}
def setupScheduler(confs: (String, String)*): TaskSchedulerImpl = {
- val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite")
+ setupSchedulerWithMaster("local", confs: _*)
+ }
+
+ def setupSchedulerWithMaster(master: String, confs: (String, String)*): TaskSchedulerImpl = {
+ val conf = new SparkConf().setMaster(master).setAppName("TaskSchedulerImplSuite")
confs.foreach { case (k, v) => conf.set(k, v) }
sc = new SparkContext(conf)
taskScheduler = new TaskSchedulerImpl(sc)
setupHelper()
}
- def setupSchedulerWithMockTaskSetBlacklist(): TaskSchedulerImpl = {
+ def setupSchedulerWithMockTaskSetBlacklist(confs: (String, String)*): TaskSchedulerImpl = {
blacklist = mock[BlacklistTracker]
val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite")
conf.set(config.BLACKLIST_ENABLED, true)
+ confs.foreach { case (k, v) => conf.set(k, v) }
+
sc = new SparkContext(conf)
taskScheduler =
new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4)) {
@@ -197,28 +205,39 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
// Even if one of the task sets has not-serializable tasks, the other task set should
// still be processed without error
taskScheduler.submitTasks(FakeTask.createTaskSet(1))
- taskScheduler.submitTasks(taskSet)
+ val taskSet2 = new TaskSet(
+ Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 1, 0, 0, null)
+ taskScheduler.submitTasks(taskSet2)
taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten
assert(taskDescriptions.map(_.executorId) === Seq("executor0"))
}
- test("refuse to schedule concurrent attempts for the same stage (SPARK-8103)") {
+ test("concurrent attempts for the same stage only have one active taskset") {
val taskScheduler = setupScheduler()
- val attempt1 = FakeTask.createTaskSet(1, 0)
- val attempt2 = FakeTask.createTaskSet(1, 1)
+ def isTasksetZombie(taskset: TaskSet): Boolean = {
+ taskScheduler.taskSetManagerForAttempt(taskset.stageId, taskset.stageAttemptId).get.isZombie
+ }
+
+ val attempt1 = FakeTask.createTaskSet(1, stageId = 0, stageAttemptId = 0)
taskScheduler.submitTasks(attempt1)
- intercept[IllegalStateException] { taskScheduler.submitTasks(attempt2) }
+ // The first submitted taskset is active
+ assert(!isTasksetZombie(attempt1))
- // OK to submit multiple if previous attempts are all zombie
- taskScheduler.taskSetManagerForAttempt(attempt1.stageId, attempt1.stageAttemptId)
- .get.isZombie = true
+ val attempt2 = FakeTask.createTaskSet(1, stageId = 0, stageAttemptId = 1)
taskScheduler.submitTasks(attempt2)
- val attempt3 = FakeTask.createTaskSet(1, 2)
- intercept[IllegalStateException] { taskScheduler.submitTasks(attempt3) }
- taskScheduler.taskSetManagerForAttempt(attempt2.stageId, attempt2.stageAttemptId)
- .get.isZombie = true
+ // The first submitted taskset is zombie now
+ assert(isTasksetZombie(attempt1))
+ // The newly submitted taskset is active
+ assert(!isTasksetZombie(attempt2))
+
+ val attempt3 = FakeTask.createTaskSet(1, stageId = 0, stageAttemptId = 2)
taskScheduler.submitTasks(attempt3)
- assert(!failedTaskSet)
+ // The first submitted taskset remains zombie
+ assert(isTasksetZombie(attempt1))
+ // The second submitted taskset is zombie now
+ assert(isTasksetZombie(attempt2))
+ // The newly submitted taskset is active
+ assert(!isTasksetZombie(attempt3))
}
test("don't schedule more tasks after a taskset is zombie") {
@@ -226,7 +245,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
val numFreeCores = 1
val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores))
- val attempt1 = FakeTask.createTaskSet(10)
+ val attempt1 = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 0)
// submit attempt 1, offer some resources, some tasks get scheduled
taskScheduler.submitTasks(attempt1)
@@ -242,7 +261,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
assert(0 === taskDescriptions2.length)
// if we schedule another attempt for the same stage, it should get scheduled
- val attempt2 = FakeTask.createTaskSet(10, 1)
+ val attempt2 = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 1)
// submit attempt 2, offer some resources, some tasks get scheduled
taskScheduler.submitTasks(attempt2)
@@ -258,7 +277,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
val numFreeCores = 10
val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores))
- val attempt1 = FakeTask.createTaskSet(10)
+ val attempt1 = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 0)
// submit attempt 1, offer some resources, some tasks get scheduled
taskScheduler.submitTasks(attempt1)
@@ -274,7 +293,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
assert(0 === taskDescriptions2.length)
// submit attempt 2
- val attempt2 = FakeTask.createTaskSet(10, 1)
+ val attempt2 = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 1)
taskScheduler.submitTasks(attempt2)
// attempt 1 finished (this can happen even if it was marked zombie earlier -- all tasks were
@@ -466,9 +485,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
}
}
- test("abort stage when all executors are blacklisted") {
+ test("abort stage when all executors are blacklisted and we cannot acquire new executor") {
taskScheduler = setupSchedulerWithMockTaskSetBlacklist()
- val taskSet = FakeTask.createTaskSet(numTasks = 10, stageAttemptId = 0)
+ val taskSet = FakeTask.createTaskSet(numTasks = 10)
taskScheduler.submitTasks(taskSet)
val tsm = stageToMockTaskSetManager(0)
@@ -503,6 +522,185 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
verify(tsm).abort(anyString(), anyObject())
}
+ test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
+ "executor can be acquired") {
+ // set the abort timer to fail immediately
+ taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
+ config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
+
+ // We have only 1 task remaining with 1 executor
+ val taskSet = FakeTask.createTaskSet(numTasks = 1)
+ taskScheduler.submitTasks(taskSet)
+ val tsm = stageToMockTaskSetManager(0)
+
+ // submit an offer with one executor
+ val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 1)
+ )).flatten
+
+ // Fail the running task
+ val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
+ taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
+ // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
+ // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
+ // before it is launched and this fails the assertion check.
+ tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
+ when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
+ "executor0", failedTask.index)).thenReturn(true)
+
+ // make an offer on the blacklisted executor. We won't schedule anything, and set the abort
+ // timer to kick in immediately
+ assert(taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 1)
+ )).flatten.size === 0)
+ // Wait for the abort timer to kick in. Even though we configure the timeout to be 0, there is a
+ // slight delay as the abort timer is launched in a separate thread.
+ eventually(timeout(500.milliseconds)) {
+ assert(tsm.isZombie)
+ }
+ }
+
+ test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
+ taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
+ config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
+
+ // We have only 1 task remaining with 1 executor
+ val taskSet = FakeTask.createTaskSet(numTasks = 1)
+ taskScheduler.submitTasks(taskSet)
+ val tsm = stageToMockTaskSetManager(0)
+
+ // submit an offer with one executor
+ val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 1)
+ )).flatten
+
+ // Fail the running task
+ val failedTask = firstTaskAttempts.head
+ taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
+ // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
+ // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
+ // before it is launched and this fails the assertion check.
+ tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
+ when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
+ "executor0", failedTask.index)).thenReturn(true)
+
+ // make an offer on the blacklisted executor. We won't schedule anything, and set the abort
+ // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted
+ // executor and try to acquire a new one.
+ assert(taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 1)
+ )).flatten.size === 0)
+ assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm))
+ assert(!tsm.isZombie)
+
+ // Offer a new executor which should be accepted
+ assert(taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor1", "host0", 1)
+ )).flatten.size === 1)
+ assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty)
+ assert(!tsm.isZombie)
+ }
+
+ // This is to test a scenario where we have two taskSets completely blacklisted and on acquiring
+ // a new executor we don't want the abort timer for the second taskSet to expire and abort the job
+ test("SPARK-22148 abort timer should clear unschedulableTaskSetToExpiryTime for all TaskSets") {
+ taskScheduler = setupSchedulerWithMockTaskSetBlacklist()
+
+ // We have 2 taskSets with 1 task remaining in each with 1 executor completely blacklisted
+ val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0)
+ taskScheduler.submitTasks(taskSet1)
+ val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 0)
+ taskScheduler.submitTasks(taskSet2)
+ val tsm = stageToMockTaskSetManager(0)
+
+ // submit an offer with one executor
+ val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 1)
+ )).flatten
+
+ assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty)
+
+ // Fail the running task
+ val failedTask = firstTaskAttempts.head
+ taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
+ tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
+ when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
+ "executor0", failedTask.index)).thenReturn(true)
+
+ // make an offer. We will schedule the task from the second taskSet. Since a task was scheduled
+ // we do not kick off the abort timer for taskSet1
+ val secondTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 1)
+ )).flatten
+
+ assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty)
+
+ val tsm2 = stageToMockTaskSetManager(1)
+ val failedTask2 = secondTaskAttempts.head
+ taskScheduler.statusUpdate(failedTask2.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
+ tsm2.handleFailedTask(failedTask2.taskId, TaskState.FAILED, UnknownReason)
+ when(tsm2.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
+ "executor0", failedTask2.index)).thenReturn(true)
+
+ // make an offer on the blacklisted executor. We won't schedule anything, and set the abort
+ // timer for taskSet1 and taskSet2
+ assert(taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 1)
+ )).flatten.size === 0)
+ assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm))
+ assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm2))
+ assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 2)
+
+ // Offer a new executor which should be accepted
+ assert(taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor1", "host1", 1)
+ )).flatten.size === 1)
+
+ // Check if all the taskSets are cleared
+ assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty)
+
+ assert(!tsm.isZombie)
+ }
+
+ // this test is to check that we don't abort a taskSet which is not being scheduled on other
+ // executors as it is waiting on locality timeout and not being aborted because it is still not
+ // completely blacklisted.
+ test("SPARK-22148 Ensure we don't abort the taskSet if we haven't been completely blacklisted") {
+ taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
+ config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0",
+ // This is to avoid any potential flakiness in the test because of large pauses in jenkins
+ config.LOCALITY_WAIT.key -> "30s"
+ )
+
+ val preferredLocation = Seq(ExecutorCacheTaskLocation("host0", "executor0"))
+ val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0,
+ preferredLocation)
+ taskScheduler.submitTasks(taskSet1)
+
+ val tsm = stageToMockTaskSetManager(0)
+
+ // submit an offer with one executor
+ var taskAttempts = taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor0", "host0", 1)
+ )).flatten
+
+ // Fail the running task
+ val failedTask = taskAttempts.head
+ taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
+ tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
+ when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
+ "executor0", failedTask.index)).thenReturn(true)
+
+ // make an offer but we won't schedule anything yet as scheduler locality is still PROCESS_LOCAL
+ assert(taskScheduler.resourceOffers(IndexedSeq(
+ WorkerOffer("executor1", "host0", 1)
+ )).flatten.isEmpty)
+
+ assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty)
+
+ assert(!tsm.isZombie)
+ }
+
/**
* Helper for performance tests. Takes the explicitly blacklisted nodes and executors; verifies
* that the blacklists are used efficiently to ensure scheduling is not O(numPendingTasks).
@@ -713,7 +911,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
test("SPARK-16106 locality levels updated if executor added to existing host") {
val taskScheduler = setupScheduler()
- taskScheduler.submitTasks(FakeTask.createTaskSet(2, 0,
+ taskScheduler.submitTasks(FakeTask.createTaskSet(2, stageId = 0, stageAttemptId = 0,
(0 until 2).map { _ => Seq(TaskLocation("host0", "executor2")) }: _*
))
@@ -751,7 +949,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
test("scheduler checks for executors that can be expired from blacklist") {
taskScheduler = setupScheduler()
- taskScheduler.submitTasks(FakeTask.createTaskSet(1, 0))
+ taskScheduler.submitTasks(FakeTask.createTaskSet(1, stageId = 0, stageAttemptId = 0))
taskScheduler.resourceOffers(IndexedSeq(
new WorkerOffer("executor0", "host0", 1)
)).flatten
@@ -935,7 +1133,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
// you'd need the previous stage to also get restarted, and then succeed, in between each
// attempt, but that happens outside what we're mocking here.)
val zombieAttempts = (0 until 2).map { stageAttempt =>
- val attempt = FakeTask.createTaskSet(10, stageAttemptId = stageAttempt)
+ val attempt = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = stageAttempt)
taskScheduler.submitTasks(attempt)
val tsm = taskScheduler.taskSetManagerForAttempt(0, stageAttempt).get
val offers = (0 until 10).map{ idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) }
@@ -954,7 +1152,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
// we've now got 2 zombie attempts, each with 9 tasks still active. Submit the 3rd attempt for
// the stage, but this time with insufficient resources so not all tasks are active.
- val finalAttempt = FakeTask.createTaskSet(10, stageAttemptId = 2)
+ val finalAttempt = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 2)
taskScheduler.submitTasks(finalAttempt)
val finalTsm = taskScheduler.taskSetManagerForAttempt(0, 2).get
val offers = (0 until 5).map{ idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) }
@@ -1057,6 +1255,29 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
assert(3 === taskDescriptions.length)
}
+ test("SPARK-29263: barrier TaskSet can't schedule when higher prio taskset takes the slots") {
+ val taskCpus = 2
+ val taskScheduler = setupSchedulerWithMaster(
+ s"local[$taskCpus]",
+ config.CPUS_PER_TASK.key -> taskCpus.toString)
+
+ val numFreeCores = 3
+ val workerOffers = IndexedSeq(
+ new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625")),
+ new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627")),
+ new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49629")))
+ val barrier = FakeTask.createBarrierTaskSet(3, stageId = 0, stageAttemptId = 0, priority = 1)
+ val highPrio = FakeTask.createTaskSet(1, stageId = 1, stageAttemptId = 0, priority = 0)
+
+ // submit highPrio and barrier taskSet
+ taskScheduler.submitTasks(highPrio)
+ taskScheduler.submitTasks(barrier)
+ val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten
+ // it schedules the highPrio task first, and then will not have enough slots to schedule
+ // the barrier taskset
+ assert(1 === taskDescriptions.length)
+ }
+
test("cancelTasks shall kill all the running tasks and fail the stage") {
val taskScheduler = setupScheduler()
@@ -1072,7 +1293,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
}
})
- val attempt1 = FakeTask.createTaskSet(10, 0)
+ val attempt1 = FakeTask.createTaskSet(10)
taskScheduler.submitTasks(attempt1)
val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 1),
@@ -1103,7 +1324,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
}
})
- val attempt1 = FakeTask.createTaskSet(10, 0)
+ val attempt1 = FakeTask.createTaskSet(10)
taskScheduler.submitTasks(attempt1)
val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 1),
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index d264adaef90a5..93a4b1f2d8276 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -1398,7 +1398,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
assert(taskSetManager1.isZombie)
assert(taskSetManager1.runningTasks === 9)
- val taskSet2 = FakeTask.createTaskSet(10, stageAttemptId = 1)
+ val taskSet2 = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 1)
sched.submitTasks(taskSet2)
sched.resourceOffers(
(11 until 20).map { idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) })
diff --git a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala
index 78f618f8a2163..0d3611c80b8d0 100644
--- a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala
@@ -16,13 +16,16 @@
*/
package org.apache.spark.security
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream}
-import java.nio.channels.Channels
+import java.io._
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, ReadableByteChannel}
import java.nio.charset.StandardCharsets.UTF_8
import java.nio.file.Files
import java.util.{Arrays, Random, UUID}
import com.google.common.io.ByteStreams
+import org.mockito.Matchers.any
+import org.mockito.Mockito._
import org.apache.spark._
import org.apache.spark.internal.config._
@@ -164,6 +167,36 @@ class CryptoStreamUtilsSuite extends SparkFunSuite {
}
}
+ test("error handling wrapper") {
+ val wrapped = mock(classOf[ReadableByteChannel])
+ val decrypted = mock(classOf[ReadableByteChannel])
+ val errorHandler = new CryptoStreamUtils.ErrorHandlingReadableChannel(decrypted, wrapped)
+
+ when(decrypted.read(any(classOf[ByteBuffer])))
+ .thenThrow(new IOException())
+ .thenThrow(new InternalError())
+ .thenReturn(1)
+
+ val out = ByteBuffer.allocate(1)
+ intercept[IOException] {
+ errorHandler.read(out)
+ }
+ intercept[InternalError] {
+ errorHandler.read(out)
+ }
+
+ val e = intercept[IOException] {
+ errorHandler.read(out)
+ }
+ assert(e.getMessage().contains("is closed"))
+ errorHandler.close()
+
+ verify(decrypted, times(2)).read(any(classOf[ByteBuffer]))
+ verify(wrapped, never()).read(any(classOf[ByteBuffer]))
+ verify(decrypted, never()).close()
+ verify(wrapped, times(1)).close()
+ }
+
private def createConf(extra: (String, String)*): SparkConf = {
val conf = new SparkConf()
extra.foreach { case (k, v) => conf.set(k, v) }
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
index 36912441c03bd..bf5ff1002ee92 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -17,7 +17,8 @@
package org.apache.spark.serializer
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream}
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
+import java.nio.ByteBuffer
import scala.collection.JavaConverters._
import scala.collection.mutable
@@ -31,7 +32,6 @@ import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite}
import org.apache.spark.scheduler.HighlyCompressedMapStatus
import org.apache.spark.serializer.KryoTest._
import org.apache.spark.storage.BlockManagerId
-import org.apache.spark.util.Utils
class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext {
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
@@ -345,8 +345,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext {
val denseBlockSizes = new Array[Long](5000)
val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L)
Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes =>
- ser.serialize(
- HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes, 1))
+ ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes))
}
}
@@ -365,30 +364,6 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext {
assert(thrown.getCause.isInstanceOf[KryoException])
}
- test("SPARK-12222: deserialize RoaringBitmap throw Buffer underflow exception") {
- val dir = Utils.createTempDir()
- val tmpfile = dir.toString + "/RoaringBitmap"
- val outStream = new FileOutputStream(tmpfile)
- val output = new KryoOutput(outStream)
- val bitmap = new RoaringBitmap
- bitmap.add(1)
- bitmap.add(3)
- bitmap.add(5)
- // Ignore Kryo because it doesn't use writeObject
- bitmap.serialize(new KryoOutputObjectOutputBridge(null, output))
- output.flush()
- output.close()
-
- val inStream = new FileInputStream(tmpfile)
- val input = new KryoInput(inStream)
- val ret = new RoaringBitmap
- // Ignore Kryo because it doesn't use readObject
- ret.deserialize(new KryoInputObjectInputBridge(null, input))
- input.close()
- assert(ret == bitmap)
- Utils.deleteRecursively(dir)
- }
-
test("KryoOutputObjectOutputBridge.writeObject and KryoInputObjectInputBridge.readObject") {
val kryo = new KryoSerializer(conf).newKryo()
@@ -462,6 +437,14 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext {
testSerializerInstanceReuse(autoReset = autoReset, referenceTracking = referenceTracking)
}
}
+
+ test("SPARK-27216: test RoaringBitmap ser/dser with Kryo") {
+ val expected = new RoaringBitmap()
+ expected.add(1787)
+ val ser = new KryoSerializer(conf).newInstance()
+ val actual: RoaringBitmap = ser.deserialize(ser.serialize(expected))
+ assert(actual === expected)
+ }
}
class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSparkContext {
@@ -498,6 +481,17 @@ class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSpar
deserializationStream.close()
assert(serInstance.deserialize[Any](helloHello) === ((hello, hello)))
}
+
+ test("SPARK-25786: ByteBuffer.array -- UnsupportedOperationException") {
+ val serInstance = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance]
+ val obj = "UnsupportedOperationException"
+ val serObj = serInstance.serialize(obj)
+ val byteBuffer = ByteBuffer.allocateDirect(serObj.array().length)
+ byteBuffer.put(serObj.array())
+ byteBuffer.flip()
+ assert(serInstance.deserialize[Any](serObj) === (obj))
+ assert(serInstance.deserialize[Any](byteBuffer) === (obj))
+ }
}
class ClassLoaderTestingObject
diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
index ea80fea905340..930338e82c440 100644
--- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
@@ -154,6 +154,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), "details2"))
val jobProps = new Properties()
+ jobProps.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, "jobDescription")
jobProps.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "jobGroup")
jobProps.setProperty("spark.scheduler.pool", "schedPool")
@@ -162,7 +163,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
check[JobDataWrapper](1) { job =>
assert(job.info.jobId === 1)
assert(job.info.name === stages.last.name)
- assert(job.info.description === None)
+ assert(job.info.description === Some("jobDescription"))
assert(job.info.status === JobExecutionStatus.RUNNING)
assert(job.info.submissionTime === Some(new Date(time)))
assert(job.info.jobGroup === Some("jobGroup"))
@@ -881,12 +882,41 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
assert(dist.memoryRemaining === maxMemory - rdd2b1.memSize - rdd1b2.memSize )
}
+ // Add block1 of rdd1 back to bm 1.
+ listener.onBlockUpdated(SparkListenerBlockUpdated(
+ BlockUpdatedInfo(bm1, rdd1b1.blockId, level, rdd1b1.memSize, rdd1b1.diskSize)))
+
+ check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+ assert(exec.info.rddBlocks === 3L)
+ assert(exec.info.memoryUsed === rdd1b1.memSize + rdd1b2.memSize + rdd2b1.memSize)
+ assert(exec.info.diskUsed === rdd1b1.diskSize + rdd1b2.diskSize + rdd2b1.diskSize)
+ }
+
// Unpersist RDD1.
listener.onUnpersistRDD(SparkListenerUnpersistRDD(rdd1b1.rddId))
intercept[NoSuchElementException] {
check[RDDStorageInfoWrapper](rdd1b1.rddId) { _ => () }
}
+ // executor1 now only contains block1 from rdd2.
+ check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+ assert(exec.info.rddBlocks === 1L)
+ assert(exec.info.memoryUsed === rdd2b1.memSize)
+ assert(exec.info.diskUsed === rdd2b1.diskSize)
+ }
+
+ // Unpersist RDD2.
+ listener.onUnpersistRDD(SparkListenerUnpersistRDD(rdd2b1.rddId))
+ intercept[NoSuchElementException] {
+ check[RDDStorageInfoWrapper](rdd2b1.rddId) { _ => () }
+ }
+
+ check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+ assert(exec.info.rddBlocks === 0L)
+ assert(exec.info.memoryUsed === 0)
+ assert(exec.info.diskUsed === 0)
+ }
+
// Update a StreamBlock.
val stream1 = StreamBlockId(1, 1L)
listener.onBlockUpdated(SparkListenerBlockUpdated(
@@ -910,6 +940,24 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
intercept[NoSuchElementException] {
check[StreamBlockData](stream1.name) { _ => () }
}
+
+ // Update a BroadcastBlock.
+ val broadcast1 = BroadcastBlockId(1L)
+ listener.onBlockUpdated(SparkListenerBlockUpdated(
+ BlockUpdatedInfo(bm1, broadcast1, level, 1L, 1L)))
+
+ check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+ assert(exec.info.memoryUsed === 1L)
+ assert(exec.info.diskUsed === 1L)
+ }
+
+ // Drop a BroadcastBlock.
+ listener.onBlockUpdated(SparkListenerBlockUpdated(
+ BlockUpdatedInfo(bm1, broadcast1, StorageLevel.NONE, 1L, 1L)))
+ check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+ assert(exec.info.memoryUsed === 0)
+ assert(exec.info.diskUsed === 0)
+ }
}
test("eviction of old data") {
@@ -1245,6 +1293,73 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
assert(allJobs.head.numFailedStages == 1)
}
+ Seq(true, false).foreach { live =>
+ test(s"Total tasks in the executor summary should match total stage tasks (live = $live)") {
+
+ val testConf = if (live) {
+ conf.clone().set(LIVE_ENTITY_UPDATE_PERIOD, Long.MaxValue)
+ } else {
+ conf.clone().set(LIVE_ENTITY_UPDATE_PERIOD, -1L)
+ }
+
+ val listener = new AppStatusListener(store, testConf, live)
+
+ Seq("1", "2").foreach { execId =>
+ listener.onExecutorAdded(SparkListenerExecutorAdded(0L, execId,
+ new ExecutorInfo("host1", 1, Map.empty)))
+ }
+ val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details")
+ listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null))
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties()))
+
+ val tasks = createTasks(4, Array("1", "2"))
+ tasks.foreach { task =>
+ listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task))
+ }
+
+ time += 1
+ tasks(0).markFinished(TaskState.FINISHED, time)
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ Success, tasks(0), null))
+ time += 1
+ tasks(1).markFinished(TaskState.FINISHED, time)
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ Success, tasks(1), null))
+
+ stage.failureReason = Some("Failed")
+ listener.onStageCompleted(SparkListenerStageCompleted(stage))
+ time += 1
+ listener.onJobEnd(SparkListenerJobEnd(1, time, JobFailed(
+ new RuntimeException("Bad Executor"))))
+
+ time += 1
+ tasks(2).markFinished(TaskState.FAILED, time)
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), null))
+ time += 1
+ tasks(3).markFinished(TaskState.FAILED, time)
+ listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType",
+ ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), null))
+
+ val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info)
+ esummary.foreach { execSummary =>
+ assert(execSummary.failedTasks === 1)
+ assert(execSummary.succeededTasks === 1)
+ assert(execSummary.killedTasks === 0)
+ }
+
+ val allExecutorSummary = store.view(classOf[ExecutorSummaryWrapper]).asScala.map(_.info)
+ assert(allExecutorSummary.size === 2)
+ allExecutorSummary.foreach { allExecSummary =>
+ assert(allExecSummary.failedTasks === 1)
+ assert(allExecSummary.activeTasks === 0)
+ assert(allExecSummary.completedTasks === 1)
+ }
+ store.delete(classOf[ExecutorSummaryWrapper], "1")
+ store.delete(classOf[ExecutorSummaryWrapper], "2")
+ }
+ }
+
test("driver logs") {
val listener = new AppStatusListener(store, conf, true)
diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala
index 92f90f3d96ddf..165fdb71cc78b 100644
--- a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala
@@ -17,8 +17,7 @@
package org.apache.spark.status
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.status.api.v1.TaskMetricDistributions
+import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.util.Distribution
import org.apache.spark.util.kvstore._
@@ -77,6 +76,45 @@ class AppStatusStoreSuite extends SparkFunSuite {
assert(store.count(classOf[CachedQuantile]) === 2)
}
+ private def createLiveStore(inMemoryStore: InMemoryStore): AppStatusStore = {
+ val conf = new SparkConf()
+ val store = new ElementTrackingStore(inMemoryStore, conf)
+ val listener = new AppStatusListener(store, conf, true, None)
+ new AppStatusStore(store, listener = Some(listener))
+ }
+
+ test("SPARK-28638: only successful tasks have taskSummary when with in memory kvstore") {
+ val store = new InMemoryStore()
+ (0 until 5).foreach { i => store.write(newTaskData(i, status = "FAILED")) }
+ Seq(new AppStatusStore(store), createLiveStore(store)).foreach { appStore =>
+ val summary = appStore.taskSummary(stageId, attemptId, uiQuantiles)
+ assert(summary.size === 0)
+ }
+ }
+
+ test("SPARK-28638: summary should contain successful tasks only when with in memory kvstore") {
+ val store = new InMemoryStore()
+
+ for (i <- 0 to 5) {
+ if (i % 2 == 1) {
+ store.write(newTaskData(i, status = "FAILED"))
+ } else {
+ store.write(newTaskData(i))
+ }
+ }
+
+ Seq(new AppStatusStore(store), createLiveStore(store)).foreach { appStore =>
+ val summary = appStore.taskSummary(stageId, attemptId, uiQuantiles).get
+
+ val values = Array(0.0, 2.0, 4.0)
+
+ val dist = new Distribution(values, 0, values.length).getQuantiles(uiQuantiles.sorted)
+ dist.zip(summary.executorRunTime).foreach { case (expected, actual) =>
+ assert(expected === actual)
+ }
+ }
+ }
+
private def compareQuantiles(count: Int, quantiles: Array[Double]): Unit = {
val store = new InMemoryStore()
val values = (0 until count).map { i =>
@@ -93,12 +131,11 @@ class AppStatusStoreSuite extends SparkFunSuite {
}
}
- private def newTaskData(i: Int): TaskDataWrapper = {
+ private def newTaskData(i: Int, status: String = "SUCCESS"): TaskDataWrapper = {
new TaskDataWrapper(
- i, i, i, i, i, i, i.toString, i.toString, i.toString, i.toString, false, Nil, None,
+ i, i, i, i, i, i, i.toString, i.toString, status, i.toString, false, Nil, None,
i, i, i, i, i, i, i, i, i, i,
i, i, i, i, i, i, i, i, i, i,
i, i, i, i, stageId, attemptId)
}
-
}
diff --git a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala
index 07a7b58404c29..435b38134bd00 100644
--- a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala
+++ b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala
@@ -17,15 +17,62 @@
package org.apache.spark.status
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
+
import org.mockito.Mockito._
+import org.scalatest.Matchers._
+import org.scalatest.concurrent.Eventually
import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.status.ElementTrackingStore._
import org.apache.spark.util.kvstore._
-class ElementTrackingStoreSuite extends SparkFunSuite {
+class ElementTrackingStoreSuite extends SparkFunSuite with Eventually {
import config._
+ test("asynchronous tracking single-fire") {
+ val store = mock(classOf[KVStore])
+ val tracking = new ElementTrackingStore(store, new SparkConf()
+ .set(ASYNC_TRACKING_ENABLED, true))
+
+ var done = new AtomicBoolean(false)
+ var type1 = new AtomicInteger(0)
+ var queued0: WriteQueueResult = null
+ var queued1: WriteQueueResult = null
+ var queued2: WriteQueueResult = null
+ var queued3: WriteQueueResult = null
+
+ tracking.addTrigger(classOf[Type1], 1) { count =>
+ val count = type1.getAndIncrement()
+
+ count match {
+ case 0 =>
+ // while in the asynchronous thread, attempt to increment twice. The first should
+ // succeed, the second should be skipped
+ queued1 = tracking.write(new Type1, checkTriggers = true)
+ queued2 = tracking.write(new Type1, checkTriggers = true)
+ case 1 =>
+ // Verify that once we've started deliver again, that we can enqueue another
+ queued3 = tracking.write(new Type1, checkTriggers = true)
+ case 2 =>
+ done.set(true)
+ }
+ }
+
+ when(store.count(classOf[Type1])).thenReturn(2L)
+ queued0 = tracking.write(new Type1, checkTriggers = true)
+ eventually {
+ done.get() shouldEqual true
+ }
+
+ tracking.close(false)
+ assert(queued0 == WriteQueued)
+ assert(queued1 == WriteQueued)
+ assert(queued2 == WriteSkippedQueue)
+ assert(queued3 == WriteQueued)
+ }
+
test("tracking for multiple types") {
val store = mock(classOf[KVStore])
val tracking = new ElementTrackingStore(store, new SparkConf()
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index dbee1f60d7af0..b44cef56d24c0 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -27,8 +27,8 @@ import scala.language.{implicitConversions, postfixOps}
import scala.reflect.ClassTag
import org.apache.commons.lang3.RandomUtils
-import org.mockito.{Matchers => mc}
-import org.mockito.Mockito.{mock, times, verify, when}
+import org.mockito.{ArgumentCaptor, Matchers => mc}
+import org.mockito.Mockito.{mock, never, spy, times, verify, when}
import org.scalatest._
import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits}
import org.scalatest.concurrent.Eventually._
@@ -43,9 +43,8 @@ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
import org.apache.spark.network.client.{RpcResponseCallback, TransportClient}
import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf}
import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap}
-import org.apache.spark.network.shuffle.{BlockFetchingListener, TempFileManager}
+import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager}
import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor}
-import org.apache.spark.network.util.TransportConf
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.scheduler.LiveListenerBus
import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite}
@@ -65,9 +64,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
implicit val defaultSignaler: Signaler = ThreadSignaler
var conf: SparkConf = null
- var store: BlockManager = null
- var store2: BlockManager = null
- var store3: BlockManager = null
+ val allStores = ArrayBuffer[BlockManager]()
var rpcEnv: RpcEnv = null
var master: BlockManagerMaster = null
val securityMgr = new SecurityManager(new SparkConf(false))
@@ -105,6 +102,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, bmConf,
memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, 0)
memManager.setMemoryStore(blockManager.memoryStore)
+ allStores += blockManager
blockManager.initialize("app-id")
blockManager
}
@@ -130,9 +128,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
// need to create a SparkContext is to initialize LiveListenerBus.
sc = mock(classOf[SparkContext])
when(sc.conf).thenReturn(conf)
- master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager",
+ master = spy(new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager",
new BlockManagerMasterEndpoint(rpcEnv, true, conf,
- new LiveListenerBus(conf))), conf, true)
+ new LiveListenerBus(conf))), conf, true))
val initialize = PrivateMethod[Unit]('initialize)
SizeEstimator invokePrivate initialize()
@@ -141,18 +139,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
override def afterEach(): Unit = {
try {
conf = null
- if (store != null) {
- store.stop()
- store = null
- }
- if (store2 != null) {
- store2.stop()
- store2 = null
- }
- if (store3 != null) {
- store3.stop()
- store3 = null
- }
+ allStores.foreach(_.stop())
+ allStores.clear()
rpcEnv.shutdown()
rpcEnv.awaitTermination()
rpcEnv = null
@@ -162,6 +150,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
}
+ private def stopBlockManager(blockManager: BlockManager): Unit = {
+ allStores -= blockManager
+ blockManager.stop()
+ }
+
test("StorageLevel object caching") {
val level1 = StorageLevel(false, false, false, 3)
// this should return the same object as level1
@@ -205,7 +198,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("master + 1 manager interaction") {
- store = makeBlockManager(20000)
+ val store = makeBlockManager(20000)
val a1 = new Array[Byte](4000)
val a2 = new Array[Byte](4000)
val a3 = new Array[Byte](4000)
@@ -235,8 +228,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("master + 2 managers interaction") {
- store = makeBlockManager(2000, "exec1")
- store2 = makeBlockManager(2000, "exec2")
+ val store = makeBlockManager(2000, "exec1")
+ val store2 = makeBlockManager(2000, "exec2")
val peers = master.getPeers(store.blockManagerId)
assert(peers.size === 1, "master did not return the other manager as a peer")
@@ -251,7 +244,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("removing block") {
- store = makeBlockManager(20000)
+ val store = makeBlockManager(20000)
val a1 = new Array[Byte](4000)
val a2 = new Array[Byte](4000)
val a3 = new Array[Byte](4000)
@@ -282,14 +275,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
assert(!store.hasLocalBlock("a1-to-remove"))
master.getLocations("a1-to-remove") should have size 0
+ assertUpdateBlockInfoReportedForRemovingBlock(store, "a1-to-remove",
+ removedFromMemory = true, removedFromDisk = false)
}
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
assert(!store.hasLocalBlock("a2-to-remove"))
master.getLocations("a2-to-remove") should have size 0
+ assertUpdateBlockInfoReportedForRemovingBlock(store, "a2-to-remove",
+ removedFromMemory = true, removedFromDisk = false)
}
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
assert(store.hasLocalBlock("a3-to-remove"))
master.getLocations("a3-to-remove") should have size 0
+ assertUpdateBlockInfoNotReported(store, "a3-to-remove")
}
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
val memStatus = master.getMemoryStatus.head._2
@@ -299,7 +297,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("removing rdd") {
- store = makeBlockManager(20000)
+ val store = makeBlockManager(20000)
val a1 = new Array[Byte](4000)
val a2 = new Array[Byte](4000)
val a3 = new Array[Byte](4000)
@@ -332,7 +330,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("removing broadcast") {
- store = makeBlockManager(2000)
+ val store = makeBlockManager(2000)
val driverStore = store
val executorStore = makeBlockManager(2000, "executor")
val a1 = new Array[Byte](400)
@@ -368,16 +366,21 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
assert(!executorStore.hasLocalBlock(broadcast0BlockId))
assert(executorStore.hasLocalBlock(broadcast1BlockId))
assert(executorStore.hasLocalBlock(broadcast2BlockId))
+ assertUpdateBlockInfoReportedForRemovingBlock(executorStore, broadcast0BlockId,
+ removedFromMemory = false, removedFromDisk = true)
// nothing should be removed from the driver store
assert(driverStore.hasLocalBlock(broadcast0BlockId))
assert(driverStore.hasLocalBlock(broadcast1BlockId))
assert(driverStore.hasLocalBlock(broadcast2BlockId))
+ assertUpdateBlockInfoNotReported(driverStore, broadcast0BlockId)
// remove broadcast 0 block from the driver as well
master.removeBroadcast(0, removeFromMaster = true, blocking = true)
assert(!driverStore.hasLocalBlock(broadcast0BlockId))
assert(driverStore.hasLocalBlock(broadcast1BlockId))
+ assertUpdateBlockInfoReportedForRemovingBlock(driverStore, broadcast0BlockId,
+ removedFromMemory = false, removedFromDisk = true)
// remove broadcast 1 block from both the stores asynchronously
// and verify all broadcast 1 blocks have been removed
@@ -385,6 +388,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
assert(!driverStore.hasLocalBlock(broadcast1BlockId))
assert(!executorStore.hasLocalBlock(broadcast1BlockId))
+ assertUpdateBlockInfoReportedForRemovingBlock(driverStore, broadcast1BlockId,
+ removedFromMemory = false, removedFromDisk = true)
+ assertUpdateBlockInfoReportedForRemovingBlock(executorStore, broadcast1BlockId,
+ removedFromMemory = false, removedFromDisk = true)
}
// remove broadcast 2 from both the stores asynchronously
@@ -395,14 +402,48 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
assert(!driverStore.hasLocalBlock(broadcast2BlockId2))
assert(!executorStore.hasLocalBlock(broadcast2BlockId))
assert(!executorStore.hasLocalBlock(broadcast2BlockId2))
+ assertUpdateBlockInfoReportedForRemovingBlock(driverStore, broadcast2BlockId,
+ removedFromMemory = false, removedFromDisk = true)
+ assertUpdateBlockInfoReportedForRemovingBlock(driverStore, broadcast2BlockId2,
+ removedFromMemory = false, removedFromDisk = true)
+ assertUpdateBlockInfoReportedForRemovingBlock(executorStore, broadcast2BlockId,
+ removedFromMemory = false, removedFromDisk = true)
+ assertUpdateBlockInfoReportedForRemovingBlock(executorStore, broadcast2BlockId2,
+ removedFromMemory = false, removedFromDisk = true)
}
executorStore.stop()
driverStore.stop()
- store = null
+ }
+
+ private def assertUpdateBlockInfoReportedForRemovingBlock(
+ store: BlockManager,
+ blockId: BlockId,
+ removedFromMemory: Boolean,
+ removedFromDisk: Boolean): Unit = {
+ def assertSizeReported(captor: ArgumentCaptor[Long], expectRemoved: Boolean): Unit = {
+ assert(captor.getAllValues().size() === 1)
+ if (expectRemoved) {
+ assert(captor.getValue() > 0)
+ } else {
+ assert(captor.getValue() === 0)
+ }
+ }
+
+ val memSizeCaptor = ArgumentCaptor.forClass(classOf[Long]).asInstanceOf[ArgumentCaptor[Long]]
+ val diskSizeCaptor = ArgumentCaptor.forClass(classOf[Long]).asInstanceOf[ArgumentCaptor[Long]]
+ verify(master).updateBlockInfo(mc.eq(store.blockManagerId), mc.eq(blockId),
+ mc.eq(StorageLevel.NONE), memSizeCaptor.capture(), diskSizeCaptor.capture())
+ assertSizeReported(memSizeCaptor, removedFromMemory)
+ assertSizeReported(diskSizeCaptor, removedFromDisk)
+ }
+
+ private def assertUpdateBlockInfoNotReported(store: BlockManager, blockId: BlockId): Unit = {
+ verify(master, never()).updateBlockInfo(mc.eq(store.blockManagerId), mc.eq(blockId),
+ mc.eq(StorageLevel.NONE), mc.anyInt(), mc.anyInt())
}
test("reregistration on heart beat") {
- store = makeBlockManager(2000)
+ val store = makeBlockManager(2000)
val a1 = new Array[Byte](400)
store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
@@ -419,7 +460,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("reregistration on block update") {
- store = makeBlockManager(2000)
+ val store = makeBlockManager(2000)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
@@ -437,7 +478,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("reregistration doesn't dead lock") {
- store = makeBlockManager(2000)
+ val store = makeBlockManager(2000)
val a1 = new Array[Byte](400)
val a2 = List(new Array[Byte](400))
@@ -475,7 +516,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("correct BlockResult returned from get() calls") {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
val list1 = List(new Array[Byte](2000), new Array[Byte](2000))
val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500))
val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray)
@@ -546,27 +587,25 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
test("SPARK-9591: getRemoteBytes from another location when Exception throw") {
conf.set("spark.shuffle.io.maxRetries", "0")
- store = makeBlockManager(8000, "executor1")
- store2 = makeBlockManager(8000, "executor2")
- store3 = makeBlockManager(8000, "executor3")
+ val store = makeBlockManager(8000, "executor1")
+ val store2 = makeBlockManager(8000, "executor2")
+ val store3 = makeBlockManager(8000, "executor3")
val list1 = List(new Array[Byte](4000))
store2.putIterator(
"list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true)
store3.putIterator(
"list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true)
assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched")
- store2.stop()
- store2 = null
+ stopBlockManager(store2)
assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched")
- store3.stop()
- store3 = null
+ stopBlockManager(store3)
// Should return None instead of throwing an exception:
assert(store.getRemoteBytes("list1").isEmpty)
}
test("SPARK-14252: getOrElseUpdate should still read from remote storage") {
- store = makeBlockManager(8000, "executor1")
- store2 = makeBlockManager(8000, "executor2")
+ val store = makeBlockManager(8000, "executor1")
+ val store2 = makeBlockManager(8000, "executor2")
val list1 = List(new Array[Byte](4000))
store2.putIterator(
"list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true)
@@ -594,7 +633,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
private def testInMemoryLRUStorage(storageLevel: StorageLevel): Unit = {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
val a1 = new Array[Byte](4000)
val a2 = new Array[Byte](4000)
val a3 = new Array[Byte](4000)
@@ -613,7 +652,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("in-memory LRU for partitions of same RDD") {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
val a1 = new Array[Byte](4000)
val a2 = new Array[Byte](4000)
val a3 = new Array[Byte](4000)
@@ -632,7 +671,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("in-memory LRU for partitions of multiple RDDs") {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY)
store.putSingle(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY)
store.putSingle(rdd(1, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY)
@@ -655,7 +694,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
encryptionTest("on-disk storage") { _conf =>
- store = makeBlockManager(1200, testConf = Some(_conf))
+ val store = makeBlockManager(1200, testConf = Some(_conf))
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -695,7 +734,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
storageLevel: StorageLevel,
getAsBytes: Boolean,
testConf: SparkConf): Unit = {
- store = makeBlockManager(12000, testConf = Some(testConf))
+ val store = makeBlockManager(12000, testConf = Some(testConf))
val accessMethod =
if (getAsBytes) store.getLocalBytesAndReleaseLock else store.getSingleAndReleaseLock
val a1 = new Array[Byte](4000)
@@ -724,7 +763,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
encryptionTest("LRU with mixed storage levels") { _conf =>
- store = makeBlockManager(12000, testConf = Some(_conf))
+ val store = makeBlockManager(12000, testConf = Some(_conf))
val a1 = new Array[Byte](4000)
val a2 = new Array[Byte](4000)
val a3 = new Array[Byte](4000)
@@ -746,7 +785,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
encryptionTest("in-memory LRU with streams") { _conf =>
- store = makeBlockManager(12000, testConf = Some(_conf))
+ val store = makeBlockManager(12000, testConf = Some(_conf))
val list1 = List(new Array[Byte](2000), new Array[Byte](2000))
val list2 = List(new Array[Byte](2000), new Array[Byte](2000))
val list3 = List(new Array[Byte](2000), new Array[Byte](2000))
@@ -774,7 +813,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
encryptionTest("LRU with mixed storage levels and streams") { _conf =>
- store = makeBlockManager(12000, testConf = Some(_conf))
+ val store = makeBlockManager(12000, testConf = Some(_conf))
val list1 = List(new Array[Byte](2000), new Array[Byte](2000))
val list2 = List(new Array[Byte](2000), new Array[Byte](2000))
val list3 = List(new Array[Byte](2000), new Array[Byte](2000))
@@ -827,7 +866,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("overly large block") {
- store = makeBlockManager(5000)
+ val store = makeBlockManager(5000)
store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY)
assert(store.getSingleAndReleaseLock("a1") === None, "a1 was in store")
store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK)
@@ -838,13 +877,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
test("block compression") {
try {
conf.set("spark.shuffle.compress", "true")
- store = makeBlockManager(20000, "exec1")
+ var store = makeBlockManager(20000, "exec1")
store.putSingle(
ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100,
"shuffle_0_0_0 was not compressed")
- store.stop()
- store = null
+ stopBlockManager(store)
conf.set("spark.shuffle.compress", "false")
store = makeBlockManager(20000, "exec2")
@@ -852,8 +890,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER)
assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000,
"shuffle_0_0_0 was compressed")
- store.stop()
- store = null
+ stopBlockManager(store)
conf.set("spark.broadcast.compress", "true")
store = makeBlockManager(20000, "exec3")
@@ -861,37 +898,32 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER)
assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000,
"broadcast_0 was not compressed")
- store.stop()
- store = null
+ stopBlockManager(store)
conf.set("spark.broadcast.compress", "false")
store = makeBlockManager(20000, "exec4")
store.putSingle(
BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER)
assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed")
- store.stop()
- store = null
+ stopBlockManager(store)
conf.set("spark.rdd.compress", "true")
store = makeBlockManager(20000, "exec5")
store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER)
assert(store.memoryStore.getSize(rdd(0, 0)) <= 1000, "rdd_0_0 was not compressed")
- store.stop()
- store = null
+ stopBlockManager(store)
conf.set("spark.rdd.compress", "false")
store = makeBlockManager(20000, "exec6")
store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER)
assert(store.memoryStore.getSize(rdd(0, 0)) >= 10000, "rdd_0_0 was compressed")
- store.stop()
- store = null
+ stopBlockManager(store)
// Check that any other block types are also kept uncompressed
store = makeBlockManager(20000, "exec7")
store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY)
assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed")
- store.stop()
- store = null
+ stopBlockManager(store)
} finally {
System.clearProperty("spark.shuffle.compress")
System.clearProperty("spark.broadcast.compress")
@@ -905,7 +937,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1)
val memoryManager = UnifiedMemoryManager(conf, numCores = 1)
val serializerManager = new SerializerManager(new JavaSerializer(conf), conf)
- store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master,
+ val store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master,
serializerManager, conf, memoryManager, mapOutputTracker,
shuffleManager, transfer, securityMgr, 0)
memoryManager.setMemoryStore(store.memoryStore)
@@ -927,7 +959,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
test("turn off updated block statuses") {
val conf = new SparkConf()
conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, false)
- store = makeBlockManager(12000, testConf = Some(conf))
+ val store = makeBlockManager(12000, testConf = Some(conf))
store.registerTask(0)
val list = List.fill(2)(new Array[Byte](2000))
@@ -955,7 +987,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
test("updated block statuses") {
val conf = new SparkConf()
conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, true)
- store = makeBlockManager(12000, testConf = Some(conf))
+ val store = makeBlockManager(12000, testConf = Some(conf))
store.registerTask(0)
val list = List.fill(2)(new Array[Byte](2000))
val bigList = List.fill(8)(new Array[Byte](2000))
@@ -1053,7 +1085,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("query block statuses") {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
val list = List.fill(2)(new Array[Byte](2000))
// Tell master. By LRU, only list2 and list3 remains.
@@ -1098,7 +1130,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("get matching blocks") {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
val list = List.fill(2)(new Array[Byte](100))
// insert some blocks
@@ -1142,7 +1174,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY)
store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY)
// Access rdd_1_0 to ensure it's not least recently used.
@@ -1156,7 +1188,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("safely unroll blocks through putIterator (disk)") {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
val memoryStore = store.memoryStore
val diskStore = store.diskStore
val smallList = List.fill(40)(new Array[Byte](100))
@@ -1195,7 +1227,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("read-locked blocks cannot be evicted from memory") {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
val arr = new Array[Byte](4000)
// First store a1 and a2, both in memory, and a3, on disk only
store.putSingle("a1", arr, StorageLevel.MEMORY_ONLY_SER)
@@ -1221,7 +1253,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
private def testReadWithLossOfOnDiskFiles(
storageLevel: StorageLevel,
readMethod: BlockManager => Option[_]): Unit = {
- store = makeBlockManager(12000)
+ val store = makeBlockManager(12000)
assert(store.putSingle("blockId", new Array[Byte](4000), storageLevel))
assert(store.getStatus("blockId").isDefined)
// Directly delete all files from the disk store, triggering failures when reading blocks:
@@ -1261,7 +1293,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") {
val mockBlockTransferService =
new MockBlockTransferService(conf.getInt("spark.block.failures.beforeLocationRefresh", 5))
- store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService))
+ val store =
+ makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService))
store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true)
assert(store.getRemoteBytes("item").isEmpty)
}
@@ -1281,7 +1314,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(
blockManagerIds)
- store = makeBlockManager(8000, "executor1", mockBlockManagerMaster,
+ val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster,
transferService = Option(mockBlockTransferService))
val block = store.getRemoteBytes("item")
.asInstanceOf[Option[ByteBuffer]]
@@ -1302,8 +1335,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
throw new InterruptedException("Intentional interrupt")
}
}
- store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService))
- store2 = makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService))
+ val store =
+ makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService))
+ val store2 =
+ makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService))
intercept[InterruptedException] {
store.putSingle("item", "value", StorageLevel.MEMORY_ONLY_2, tellMaster = true)
}
@@ -1313,8 +1348,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
}
test("SPARK-17484: master block locations are updated following an invalid remote block fetch") {
- store = makeBlockManager(8000, "executor1")
- store2 = makeBlockManager(8000, "executor2")
+ val store = makeBlockManager(8000, "executor1")
+ val store2 = makeBlockManager(8000, "executor2")
store.putSingle("item", "value", StorageLevel.MEMORY_ONLY, tellMaster = true)
assert(master.getLocations("item").nonEmpty)
store.removeBlock("item", tellMaster = false)
@@ -1411,7 +1446,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
Option(BlockLocationsAndStatus(blockLocations, blockStatus)))
when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockLocations)
- store = makeBlockManager(8000, "executor1", mockBlockManagerMaster,
+ val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster,
transferService = Option(mockBlockTransferService))
val block = store.getRemoteBytes("item")
.asInstanceOf[Option[ByteBuffer]]
@@ -1437,7 +1472,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService {
var numCalls = 0
- var tempFileManager: TempFileManager = null
+ var tempFileManager: DownloadFileManager = null
override def init(blockDataManager: BlockDataManager): Unit = {}
@@ -1447,7 +1482,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
execId: String,
blockIds: Array[String],
listener: BlockFetchingListener,
- tempFileManager: TempFileManager): Unit = {
+ tempFileManager: DownloadFileManager): Unit = {
listener.onBlockFetchSuccess("mockBlockId", new NioManagedBuffer(ByteBuffer.allocate(1)))
}
@@ -1474,7 +1509,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
port: Int,
execId: String,
blockId: String,
- tempFileManager: TempFileManager): ManagedBuffer = {
+ tempFileManager: DownloadFileManager): ManagedBuffer = {
numCalls += 1
this.tempFileManager = tempFileManager
if (numCalls <= maxFailures) {
diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
index a2997dbd1b1ac..b268195e09a5b 100644
--- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
@@ -33,7 +33,7 @@ import org.scalatest.PrivateMethodTester
import org.apache.spark.{SparkFunSuite, TaskContext}
import org.apache.spark.network._
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
-import org.apache.spark.network.shuffle.{BlockFetchingListener, TempFileManager}
+import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager}
import org.apache.spark.network.util.LimitedInputStream
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.util.Utils
@@ -478,12 +478,12 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
val remoteBlocks = Map[BlockId, ManagedBuffer](
ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer())
val transfer = mock(classOf[BlockTransferService])
- var tempFileManager: TempFileManager = null
+ var tempFileManager: DownloadFileManager = null
when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
.thenAnswer(new Answer[Unit] {
override def answer(invocation: InvocationOnMock): Unit = {
val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
- tempFileManager = invocation.getArguments()(5).asInstanceOf[TempFileManager]
+ tempFileManager = invocation.getArguments()(5).asInstanceOf[DownloadFileManager]
Future {
listener.onBlockFetchSuccess(
ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
index e86cadfeebcff..8eef67eb5ac9d 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -39,7 +39,7 @@ import org.apache.spark._
import org.apache.spark.LocalSparkContext._
import org.apache.spark.api.java.StorageLevels
import org.apache.spark.deploy.history.HistoryServerSuite
-import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE
+import org.apache.spark.internal.config.{EXECUTOR_HEARTBEAT_INTERVAL, MEMORY_OFFHEAP_SIZE}
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.status.api.v1.{JacksonMessageWriter, RDDDataDistribution, StageStatus}
import org.apache.spark.status.config._
@@ -99,14 +99,18 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B
* Create a test SparkContext with the SparkUI enabled.
* It is safe to `get` the SparkUI directly from the SparkContext returned here.
*/
- private def newSparkContext(killEnabled: Boolean = true): SparkContext = {
+ private def newSparkContext(
+ killEnabled: Boolean = true,
+ master: String = "local",
+ additionalConfs: Map[String, String] = Map.empty): SparkContext = {
val conf = new SparkConf()
- .setMaster("local")
+ .setMaster(master)
.setAppName("test")
.set("spark.ui.enabled", "true")
.set("spark.ui.port", "0")
.set("spark.ui.killEnabled", killEnabled.toString)
.set(MEMORY_OFFHEAP_SIZE.key, "64m")
+ additionalConfs.foreach { case (k, v) => conf.set(k, v) }
val sc = new SparkContext(conf)
assert(sc.ui.isDefined)
sc
@@ -724,6 +728,31 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B
}
}
+ test("Staleness of Spark UI should not last minutes or hours") {
+ withSpark(newSparkContext(
+ master = "local[2]",
+ // Set a small heart beat interval to make the test fast
+ additionalConfs = Map(
+ EXECUTOR_HEARTBEAT_INTERVAL.key -> "10ms",
+ LIVE_ENTITY_UPDATE_MIN_FLUSH_PERIOD.key -> "10ms"))) { sc =>
+ sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "true")
+ val f = sc.parallelize(1 to 1000, 1000).foreachAsync { _ =>
+ // Make the task never finish so there won't be any task start/end events after the first 2
+ // tasks start.
+ Thread.sleep(300000)
+ }
+ try {
+ eventually(timeout(10.seconds)) {
+ val jobsJson = getJson(sc.ui.get, "jobs")
+ jobsJson.children.length should be (1)
+ (jobsJson.children.head \ "numActiveTasks").extract[Int] should be (2)
+ }
+ } finally {
+ f.cancel()
+ }
+ }
+ }
+
def goToUi(sc: SparkContext, path: String): Unit = {
goToUi(sc.ui.get, path)
}
diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala
index cdc7f541b9552..06f01a60868f9 100644
--- a/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala
@@ -81,19 +81,19 @@ class StoragePageSuite extends SparkFunSuite {
Seq("1", "rdd1", "Memory Deserialized 1x Replicated", "10", "100%", "100.0 B", "0.0 B"))
// Check the url
assert(((xmlNodes \\ "tr")(0) \\ "td" \ "a")(0).attribute("href").map(_.text) ===
- Some("http://localhost:4040/storage/rdd?id=1"))
+ Some("http://localhost:4040/storage/rdd/?id=1"))
assert(((xmlNodes \\ "tr")(1) \\ "td").map(_.text.trim) ===
Seq("2", "rdd2", "Disk Serialized 1x Replicated", "5", "50%", "0.0 B", "200.0 B"))
// Check the url
assert(((xmlNodes \\ "tr")(1) \\ "td" \ "a")(0).attribute("href").map(_.text) ===
- Some("http://localhost:4040/storage/rdd?id=2"))
+ Some("http://localhost:4040/storage/rdd/?id=2"))
assert(((xmlNodes \\ "tr")(2) \\ "td").map(_.text.trim) ===
Seq("3", "rdd3", "Disk Memory Serialized 1x Replicated", "10", "100%", "400.0 B", "500.0 B"))
// Check the url
assert(((xmlNodes \\ "tr")(2) \\ "td" \ "a")(0).attribute("href").map(_.text) ===
- Some("http://localhost:4040/storage/rdd?id=3"))
+ Some("http://localhost:4040/storage/rdd/?id=3"))
}
test("empty rddTable") {
diff --git a/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala
index 688fcd9f9aaba..29421f7aa9e36 100644
--- a/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala
@@ -17,6 +17,9 @@
package org.apache.spark.util
+import java.lang.ref.PhantomReference
+import java.lang.ref.ReferenceQueue
+
import org.apache.spark.SparkFunSuite
class CompletionIteratorSuite extends SparkFunSuite {
@@ -44,4 +47,23 @@ class CompletionIteratorSuite extends SparkFunSuite {
assert(!completionIter.hasNext)
assert(numTimesCompleted === 1)
}
+ test("reference to sub iterator should not be available after completion") {
+ var sub = Iterator(1, 2, 3)
+
+ val refQueue = new ReferenceQueue[Iterator[Int]]
+ val ref = new PhantomReference[Iterator[Int]](sub, refQueue)
+
+ val iter = CompletionIterator[Int, Iterator[Int]](sub, {})
+ sub = null
+ iter.toArray
+
+ for (_ <- 1 to 100 if !ref.isEnqueued) {
+ System.gc()
+ if (!ref.isEnqueued) {
+ Thread.sleep(10)
+ }
+ }
+ assert(ref.isEnqueued)
+ assert(refQueue.poll() === ref)
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala
index 52cd5378bc715..1a3e880b14d78 100644
--- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala
@@ -128,7 +128,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging {
val files = testRolling(appender, testOutputStream, textToAppend, 0, isCompressed = true)
files.foreach { file =>
logInfo(file.toString + ": " + file.length + " bytes")
- assert(file.length < rolloverSize)
+ assert(file.length <= rolloverSize)
}
}
diff --git a/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala b/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala
index f9e1b791c86ea..e6b71b062bf80 100644
--- a/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.utils
+package org.apache.spark.util
import org.apache.hadoop.fs.Path
@@ -23,7 +23,6 @@ import org.apache.spark.{SharedSparkContext, SparkContext, SparkFunSuite}
import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.util.PeriodicRDDCheckpointer
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.Utils
class PeriodicRDDCheckpointerSuite extends SparkFunSuite with SharedSparkContext {
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 418d2f9b88500..39f4fba78583f 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -1184,6 +1184,55 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
assert(Utils.getSimpleName(classOf[MalformedClassObject.MalformedClass]) ===
"UtilsSuite$MalformedClassObject$MalformedClass")
}
+
+ test("stringHalfWidth") {
+ // scalastyle:off nonascii
+ assert(Utils.stringHalfWidth(null) == 0)
+ assert(Utils.stringHalfWidth("") == 0)
+ assert(Utils.stringHalfWidth("ab c") == 4)
+ assert(Utils.stringHalfWidth("1098") == 4)
+ assert(Utils.stringHalfWidth("mø") == 2)
+ assert(Utils.stringHalfWidth("γύρ") == 3)
+ assert(Utils.stringHalfWidth("pê") == 2)
+ assert(Utils.stringHalfWidth("ー") == 2)
+ assert(Utils.stringHalfWidth("测") == 2)
+ assert(Utils.stringHalfWidth("か") == 2)
+ assert(Utils.stringHalfWidth("걸") == 2)
+ assert(Utils.stringHalfWidth("à") == 1)
+ assert(Utils.stringHalfWidth("焼") == 2)
+ assert(Utils.stringHalfWidth("羍む") == 4)
+ assert(Utils.stringHalfWidth("뺭ᾘ") == 3)
+ assert(Utils.stringHalfWidth("\u0967\u0968\u0969") == 3)
+ // scalastyle:on nonascii
+ }
+
+ test("trimExceptCRLF standalone") {
+ val crlfSet = Set("\r", "\n")
+ val nonPrintableButCRLF = (0 to 32).map(_.toChar.toString).toSet -- crlfSet
+
+ // identity for CRLF
+ crlfSet.foreach { s => Utils.trimExceptCRLF(s) === s }
+
+ // empty for other non-printables
+ nonPrintableButCRLF.foreach { s => assert(Utils.trimExceptCRLF(s) === "") }
+
+ // identity for a printable string
+ assert(Utils.trimExceptCRLF("a") === "a")
+
+ // identity for strings with CRLF
+ crlfSet.foreach { s =>
+ assert(Utils.trimExceptCRLF(s"${s}a") === s"${s}a")
+ assert(Utils.trimExceptCRLF(s"a${s}") === s"a${s}")
+ assert(Utils.trimExceptCRLF(s"b${s}b") === s"b${s}b")
+ }
+
+ // trim nonPrintableButCRLF except when inside a string
+ nonPrintableButCRLF.foreach { s =>
+ assert(Utils.trimExceptCRLF(s"${s}a") === "a")
+ assert(Utils.trimExceptCRLF(s"a${s}") === "a")
+ assert(Utils.trimExceptCRLF(s"b${s}b") === s"b${s}b")
+ }
+ }
}
private class SimpleExtension
diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala
index d542ba0b6640d..cd25265784136 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala
@@ -17,9 +17,8 @@
package org.apache.spark.util.collection
-import java.util.Objects
-
import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration._
import scala.ref.WeakReference
import org.scalatest.Matchers
@@ -459,7 +458,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite
// https://github.com/scala/scala/blob/2.13.x/test/junit/scala/tools/testing/AssertUtil.scala
// (lines 69-89)
// assert(map.currentMap == null)
- eventually {
+ eventually(timeout(5 seconds), interval(200 milliseconds)) {
System.gc()
// direct asserts introduced some macro generated code that held a reference to the map
val tmpIsNull = null == underlyingMapRef.get.orNull
@@ -509,7 +508,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite
.sorted
assert(it.isEmpty)
- assert(keys == (0 until 100))
+ assert(keys == (0 until 100).toList)
assert(map.numSpills == 0)
// these asserts try to show that we're no longer holding references to the underlying map.
diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala
index 3e56db5ea116a..47173b89e91e2 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala
@@ -26,7 +26,7 @@ import org.apache.spark._
import org.apache.spark.memory.MemoryTestingUtils
import org.apache.spark.serializer.{JavaSerializer, KryoSerializer}
import org.apache.spark.unsafe.array.LongArray
-import org.apache.spark.unsafe.memory.OnHeapMemoryBlock
+import org.apache.spark.unsafe.memory.MemoryBlock
import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordPointerAndKeyPrefix, UnsafeSortDataFormat}
class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext {
@@ -105,8 +105,9 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext {
// the form [150000000, 150000001, 150000002, ...., 300000000, 0, 1, 2, ..., 149999999]
// that can trigger copyRange() in TimSort.mergeLo() or TimSort.mergeHi()
val ref = Array.tabulate[Long](size) { i => if (i < size / 2) size / 2 + i else i }
- val buf = new LongArray(OnHeapMemoryBlock.fromArray(ref))
- val tmpBuf = new LongArray(new OnHeapMemoryBlock((size/2) * 8L))
+ val buf = new LongArray(MemoryBlock.fromLongArray(ref))
+ val tmp = new Array[Long](size/2)
+ val tmpBuf = new LongArray(MemoryBlock.fromLongArray(tmp))
new Sorter(new UnsafeSortDataFormat(tmpBuf)).sort(
buf, 0, size, new Comparator[RecordPointerAndKeyPrefix] {
diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala
index 151235dd0fb90..68bcc5e5a5092 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala
@@ -185,16 +185,6 @@ class OpenHashMapSuite extends SparkFunSuite with Matchers {
assert(map.contains(null))
}
- test("support for more than 12M items") {
- val cnt = 12000000 // 12M
- val map = new OpenHashMap[Int, Int](cnt)
- for (i <- 0 until cnt) {
- map(i) = 1
- }
- val numInvalidValues = map.iterator.count(_._2 == 0)
- assertResult(0)(numInvalidValues)
- }
-
test("distinguish between the 0/0.0/0L and null") {
val specializedMap1 = new OpenHashMap[String, Long]
specializedMap1("a") = null.asInstanceOf[Long]
diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala
index b887f937a9da9..44d2118d77945 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala
@@ -255,4 +255,17 @@ class OpenHashSetSuite extends SparkFunSuite with Matchers {
val set = new OpenHashSet[Long](0)
assert(set.size === 0)
}
+
+ test("support for more than 12M items") {
+ val cnt = 12000000 // 12M
+ val set = new OpenHashSet[Int](cnt)
+ for (i <- 0 until cnt) {
+ set.add(i)
+ assert(set.contains(i))
+
+ val pos1 = set.getPos(i)
+ val pos2 = set.addWithoutResize(i) & OpenHashSet.POSITION_MASK
+ assert(pos1 == pos2)
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala
index 73546ef1b7a60..38cb37c524594 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala
@@ -125,6 +125,7 @@ class PrefixComparatorsSuite extends SparkFunSuite with PropertyChecks {
val nan2Prefix = PrefixComparators.DoublePrefixComparator.computePrefix(nan2)
assert(nan1Prefix === nan2Prefix)
val doubleMaxPrefix = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MaxValue)
+ // NaN is greater than the max double value.
assert(PrefixComparators.DOUBLE.compare(nan1Prefix, doubleMaxPrefix) === 1)
}
@@ -134,22 +135,34 @@ class PrefixComparatorsSuite extends SparkFunSuite with PropertyChecks {
assert(java.lang.Double.doubleToRawLongBits(negativeNan) < 0)
val prefix = PrefixComparators.DoublePrefixComparator.computePrefix(negativeNan)
val doubleMaxPrefix = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MaxValue)
+ // -NaN is greater than the max double value.
assert(PrefixComparators.DOUBLE.compare(prefix, doubleMaxPrefix) === 1)
}
test("double prefix comparator handles other special values properly") {
- val nullValue = 0L
+ // See `SortPrefix.nullValue` for how we deal with nulls for float/double type
+ val smallestNullPrefix = 0L
+ val largestNullPrefix = -1L
val nan = PrefixComparators.DoublePrefixComparator.computePrefix(Double.NaN)
val posInf = PrefixComparators.DoublePrefixComparator.computePrefix(Double.PositiveInfinity)
val negInf = PrefixComparators.DoublePrefixComparator.computePrefix(Double.NegativeInfinity)
val minValue = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MinValue)
val maxValue = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MaxValue)
val zero = PrefixComparators.DoublePrefixComparator.computePrefix(0.0)
+ val minusZero = PrefixComparators.DoublePrefixComparator.computePrefix(-0.0)
+
+ // null is greater than everything including NaN, when we need to treat it as the largest value.
+ assert(PrefixComparators.DOUBLE.compare(largestNullPrefix, nan) === 1)
+ // NaN is greater than the positive infinity.
assert(PrefixComparators.DOUBLE.compare(nan, posInf) === 1)
assert(PrefixComparators.DOUBLE.compare(posInf, maxValue) === 1)
assert(PrefixComparators.DOUBLE.compare(maxValue, zero) === 1)
assert(PrefixComparators.DOUBLE.compare(zero, minValue) === 1)
assert(PrefixComparators.DOUBLE.compare(minValue, negInf) === 1)
- assert(PrefixComparators.DOUBLE.compare(negInf, nullValue) === 1)
+ // null is smaller than everything including negative infinity, when we need to treat it as
+ // the smallest value.
+ assert(PrefixComparators.DOUBLE.compare(negInf, smallestNullPrefix) === 1)
+ // 0.0 should be equal to -0.0.
+ assert(PrefixComparators.DOUBLE.compare(zero, minusZero) === 0)
}
}
diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala
index ddf3740e76a7a..d5956ea32096a 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala
@@ -27,7 +27,7 @@ import com.google.common.primitives.Ints
import org.apache.spark.SparkFunSuite
import org.apache.spark.internal.Logging
import org.apache.spark.unsafe.array.LongArray
-import org.apache.spark.unsafe.memory.OnHeapMemoryBlock
+import org.apache.spark.unsafe.memory.MemoryBlock
import org.apache.spark.util.collection.Sorter
import org.apache.spark.util.random.XORShiftRandom
@@ -78,14 +78,14 @@ class RadixSortSuite extends SparkFunSuite with Logging {
private def generateTestData(size: Long, rand: => Long): (Array[JLong], LongArray) = {
val ref = Array.tabulate[Long](Ints.checkedCast(size)) { i => rand }
val extended = ref ++ Array.fill[Long](Ints.checkedCast(size))(0)
- (ref.map(i => new JLong(i)), new LongArray(OnHeapMemoryBlock.fromArray(extended)))
+ (ref.map(i => new JLong(i)), new LongArray(MemoryBlock.fromLongArray(extended)))
}
private def generateKeyPrefixTestData(size: Long, rand: => Long): (LongArray, LongArray) = {
val ref = Array.tabulate[Long](Ints.checkedCast(size * 2)) { i => rand }
val extended = ref ++ Array.fill[Long](Ints.checkedCast(size * 2))(0)
- (new LongArray(OnHeapMemoryBlock.fromArray(ref)),
- new LongArray(OnHeapMemoryBlock.fromArray(extended)))
+ (new LongArray(MemoryBlock.fromLongArray(ref)),
+ new LongArray(MemoryBlock.fromLongArray(extended)))
}
private def collectToArray(array: LongArray, offset: Int, length: Long): Array[Long] = {
@@ -110,7 +110,7 @@ class RadixSortSuite extends SparkFunSuite with Logging {
}
private def referenceKeyPrefixSort(buf: LongArray, lo: Long, hi: Long, refCmp: PrefixComparator) {
- val sortBuffer = new LongArray(new OnHeapMemoryBlock(buf.size() * 8L))
+ val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt)))
new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort(
buf, Ints.checkedCast(lo), Ints.checkedCast(hi), new Comparator[RecordPointerAndKeyPrefix] {
override def compare(
diff --git a/dev/.rat-excludes b/dev/.rat-excludes
index 466135e72233a..0370d5f85cce2 100644
--- a/dev/.rat-excludes
+++ b/dev/.rat-excludes
@@ -22,7 +22,7 @@ docs
slaves
spark-env.cmd
bootstrap-tooltip.js
-jquery-1.11.1.min.js
+jquery-1.12.4.min.js
d3.min.js
dagre-d3.min.js
graphlib-dot.min.js
@@ -34,8 +34,8 @@ dataTables.bootstrap.min.js
dataTables.rowsGroup.js
jquery.blockUI.min.js
jquery.cookies.2.2.0.min.js
-jquery.dataTables.1.10.4.min.css
-jquery.dataTables.1.10.4.min.js
+jquery.dataTables.1.10.18.min.css
+jquery.dataTables.1.10.18.min.js
jquery.mustache.js
jsonFormatter.min.css
jsonFormatter.min.js
@@ -110,4 +110,5 @@ spark-warehouse
structured-streaming/*
kafka-source-initial-offset-version-2.1.0.bin
kafka-source-initial-offset-future-version.bin
+announce.tmpl
vote.tmpl
diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1
index 8a04b621f8ce4..df64193c18866 100644
--- a/dev/appveyor-install-dependencies.ps1
+++ b/dev/appveyor-install-dependencies.ps1
@@ -90,7 +90,7 @@ Invoke-Expression "7z.exe x maven.zip"
# add maven to environment variables
$env:Path += ";$tools\apache-maven-$mavenVer\bin"
$env:M2_HOME = "$tools\apache-maven-$mavenVer"
-$env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=512m"
+$env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=1g"
Pop-Location
diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml
index bbda824dd13b4..945686de49967 100644
--- a/dev/checkstyle-suppressions.xml
+++ b/dev/checkstyle-suppressions.xml
@@ -17,7 +17,7 @@
+"https://checkstyle.org/dtds/suppressions_1_1.dtd">
\n?', re.DOTALL), '', pr["body"]).lstrip()
+ if modified_body != pr["body"]:
+ print("=" * 80)
+ print(modified_body)
+ print("=" * 80)
+ print("I've removed the comments from PR template like the above:")
+ result = input("Would you like to use the modified body? (y/n): ")
+ if result.lower() == "y":
+ body = modified_body
+ print("Using modified body:")
+ else:
+ body = pr["body"]
+ print("Using original body:")
+ print("=" * 80)
+ print(body)
+ print("=" * 80)
+ else:
+ body = pr["body"]
target_ref = pr["base"]["ref"]
user_login = pr["user"]["login"]
base_ref = pr["head"]["ref"]
diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py
index e6fe3b82ed202..eca88f2391bf8 100755
--- a/dev/run-tests-jenkins.py
+++ b/dev/run-tests-jenkins.py
@@ -115,7 +115,8 @@ def run_tests(tests_timeout):
os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait()
failure_note_by_errcode = {
- 1: 'executing the `dev/run-tests` script', # error to denote run-tests script failures
+ # error to denote run-tests script failures:
+ 1: 'executing the `dev/run-tests` script',
ERROR_CODES["BLOCK_GENERAL"]: 'some tests',
ERROR_CODES["BLOCK_RAT"]: 'RAT tests',
ERROR_CODES["BLOCK_SCALA_STYLE"]: 'Scala style tests',
@@ -130,7 +131,7 @@ def run_tests(tests_timeout):
ERROR_CODES["BLOCK_PYSPARK_UNIT_TESTS"]: 'PySpark unit tests',
ERROR_CODES["BLOCK_PYSPARK_PIP_TESTS"]: 'PySpark pip packaging tests',
ERROR_CODES["BLOCK_SPARKR_UNIT_TESTS"]: 'SparkR unit tests',
- ERROR_CODES["BLOCK_TIMEOUT"]: 'from timeout after a configured wait of \`%s\`' % (
+ ERROR_CODES["BLOCK_TIMEOUT"]: 'from timeout after a configured wait of `%s`' % (
tests_timeout)
}
diff --git a/dev/run-tests.py b/dev/run-tests.py
index d9d3789ac1255..5915b52c8ef5c 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -42,15 +42,20 @@ def determine_modules_for_files(filenames):
"""
Given a list of filenames, return the set of modules that contain those files.
If a file is not associated with a more specific submodule, then this method will consider that
- file to belong to the 'root' module.
+ file to belong to the 'root' module. GitHub Action and Appveyor files are ignored.
>>> sorted(x.name for x in determine_modules_for_files(["python/pyspark/a.py", "sql/core/foo"]))
['pyspark-core', 'sql']
>>> [x.name for x in determine_modules_for_files(["file_not_matched_by_any_subproject"])]
['root']
+ >>> [x.name for x in determine_modules_for_files( \
+ [".github/workflows/master.yml", "appveyor.yml"])]
+ []
"""
changed_modules = set()
for filename in filenames:
+ if filename in (".github/workflows/master.yml", "appveyor.yml"):
+ continue
matched_at_least_one_module = False
for module in modules.all_modules:
if module.contains_file(filename):
@@ -169,7 +174,7 @@ def determine_java_version(java_exe):
# find raw version string, eg 'java version "1.8.0_25"'
raw_version_str = next(x for x in raw_output_lines if " version " in x)
- match = re.search('(\d+)\.(\d+)\.(\d+)', raw_version_str)
+ match = re.search(r'(\d+)\.(\d+)\.(\d+)', raw_version_str)
major = int(match.group(1))
minor = int(match.group(2))
@@ -249,15 +254,6 @@ def get_zinc_port():
return random.randrange(3030, 4030)
-def kill_zinc_on_port(zinc_port):
- """
- Kill the Zinc process running on the given port, if one exists.
- """
- cmd = "%s -P |grep %s | grep LISTEN | awk '{ print $2; }' | xargs kill"
- lsof_exe = which("lsof")
- subprocess.check_call(cmd % (lsof_exe if lsof_exe else "/usr/sbin/lsof", zinc_port), shell=True)
-
-
def exec_maven(mvn_args=()):
"""Will call Maven in the current directory with the list of mvn_args passed
in and returns the subprocess for any further processing"""
@@ -267,7 +263,6 @@ def exec_maven(mvn_args=()):
zinc_flag = "-DzincPort=%s" % zinc_port
flags = [os.path.join(SPARK_HOME, "build", "mvn"), "--force", zinc_flag]
run_cmd(flags + mvn_args)
- kill_zinc_on_port(zinc_port)
def exec_sbt(sbt_args=()):
diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index 2aa355504bf29..26905279e4bb2 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -252,6 +252,9 @@ def __hash__(self):
)
+# kafka-0-8 support is deprecated as of Spark 2.3 and removed in Spark 3.x.
+# Since Spark 2.4 supports Scala-2.12, and kafka-0-8 does not, we have made
+# streaming-kafka-0-8 optional for pyspark testing in 2.4.
streaming_kafka = Module(
name="streaming-kafka-0-8",
dependencies=[streaming],
@@ -387,6 +390,8 @@ def __hash__(self):
"pyspark.profiler",
"pyspark.shuffle",
"pyspark.tests",
+ "pyspark.test_broadcast",
+ "pyspark.test_serializers",
"pyspark.util",
]
)
@@ -555,6 +560,16 @@ def __hash__(self):
sbt_test_goals=["kubernetes/test"]
)
+
+spark_ganglia_lgpl = Module(
+ name="spark-ganglia-lgpl",
+ dependencies=[],
+ build_profile_flags=["-Pspark-ganglia-lgpl"],
+ source_file_regexes=[
+ "external/spark-ganglia-lgpl",
+ ]
+)
+
# 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/dev/test-dependencies.sh b/dev/test-dependencies.sh
index 2fbd6b5e98f7f..5a4a8940071fd 100755
--- a/dev/test-dependencies.sh
+++ b/dev/test-dependencies.sh
@@ -29,7 +29,6 @@ export LC_ALL=C
# TODO: This would be much nicer to do in SBT, once SBT supports Maven-style resolution.
# NOTE: These should match those in the release publishing script
-HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Pflume -Phive"
MVN="build/mvn"
HADOOP_PROFILES=(
hadoop-2.6
@@ -37,6 +36,16 @@ HADOOP_PROFILES=(
hadoop-3.1
)
+SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version $@ 2>/dev/null\
+ | grep -v "INFO"\
+ | grep -v "WARNING"\
+ | tail -n 1)
+if [ "$SCALA_VERSION" = "2.11" ]; then
+ HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Pflume -Phive"
+else
+ HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkubernetes -Pyarn -Pflume -Phive"
+fi
+
# We'll switch the version to a temp. one, publish POMs using that new version, then switch back to
# the old version. We need to do this because the `dependency:build-classpath` task needs to
# resolve Spark's internal submodule dependencies.
diff --git a/dev/tox.ini b/dev/tox.ini
index 28dad8f3b5c7c..6ec223b743b4e 100644
--- a/dev/tox.ini
+++ b/dev/tox.ini
@@ -14,6 +14,6 @@
# limitations under the License.
[pycodestyle]
-ignore=E402,E731,E241,W503,E226,E722,E741,E305
+ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504
max-line-length=100
exclude=cloudpickle.py,heapq3.py,shared.py,python/docs/conf.py,work/*/*.py,python/.eggs/*,dist/*
diff --git a/docs/README.md b/docs/README.md
index fb67c4b3586d6..41b1a2df2490f 100644
--- a/docs/README.md
+++ b/docs/README.md
@@ -22,9 +22,9 @@ $ sudo gem install jekyll jekyll-redirect-from pygments.rb
$ sudo pip install Pygments
# Following is needed only for generating API docs
$ sudo pip install sphinx pypandoc mkdocs
-$ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="http://cran.stat.ucla.edu/")'
-$ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="http://cran.stat.ucla.edu/")'
-$ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="http://cran.stat.ucla.edu/")'
+$ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")'
+$ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")'
+$ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")'
```
Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to replace gem with gem2.0.
diff --git a/docs/_config.yml b/docs/_config.yml
index 095fadb93fe5d..6818c50d4bb91 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -14,10 +14,10 @@ include:
# These allow the documentation to be updated with newer releases
# of Spark, Scala, and Mesos.
-SPARK_VERSION: 2.4.0-SNAPSHOT
-SPARK_VERSION_SHORT: 2.4.0
-SCALA_BINARY_VERSION: "2.11"
-SCALA_VERSION: "2.11.8"
+SPARK_VERSION: 2.4.5-SNAPSHOT
+SPARK_VERSION_SHORT: 2.4.5
+SCALA_BINARY_VERSION: "2.12"
+SCALA_VERSION: "2.12.10"
MESOS_VERSION: 1.0.0
SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK
SPARK_GITHUB_URL: https://github.com/apache/spark
diff --git a/docs/_data/menu-ml.yaml b/docs/_data/menu-ml.yaml
index b5a6641e2e7e2..8e366f7f029aa 100644
--- a/docs/_data/menu-ml.yaml
+++ b/docs/_data/menu-ml.yaml
@@ -1,5 +1,7 @@
- text: Basic statistics
url: ml-statistics.html
+- text: Data sources
+ url: ml-datasource
- text: Pipelines
url: ml-pipeline.html
- text: Extracting, transforming and selecting features
diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml
new file mode 100644
index 0000000000000..cd065ea01dda4
--- /dev/null
+++ b/docs/_data/menu-sql.yaml
@@ -0,0 +1,81 @@
+- text: Getting Started
+ url: sql-getting-started.html
+ subitems:
+ - text: "Starting Point: SparkSession"
+ url: sql-getting-started.html#starting-point-sparksession
+ - text: Creating DataFrames
+ url: sql-getting-started.html#creating-dataframes
+ - text: Untyped Dataset Operations (DataFrame operations)
+ url: sql-getting-started.html#untyped-dataset-operations-aka-dataframe-operations
+ - text: Running SQL Queries Programmatically
+ url: sql-getting-started.html#running-sql-queries-programmatically
+ - text: Global Temporary View
+ url: sql-getting-started.html#global-temporary-view
+ - text: Creating Datasets
+ url: sql-getting-started.html#creating-datasets
+ - text: Interoperating with RDDs
+ url: sql-getting-started.html#interoperating-with-rdds
+ - text: Aggregations
+ url: sql-getting-started.html#aggregations
+- text: Data Sources
+ url: sql-data-sources.html
+ subitems:
+ - text: "Generic Load/Save Functions"
+ url: sql-data-sources-load-save-functions.html
+ - text: Parquet Files
+ url: sql-data-sources-parquet.html
+ - text: ORC Files
+ url: sql-data-sources-orc.html
+ - text: JSON Files
+ url: sql-data-sources-json.html
+ - text: Hive Tables
+ url: sql-data-sources-hive-tables.html
+ - text: JDBC To Other Databases
+ url: sql-data-sources-jdbc.html
+ - text: Avro Files
+ url: sql-data-sources-avro.html
+ - text: Troubleshooting
+ url: sql-data-sources-troubleshooting.html
+- text: Performance Tuning
+ url: sql-performance-tuning.html
+ subitems:
+ - text: Caching Data In Memory
+ url: sql-performance-tuning.html#caching-data-in-memory
+ - text: Other Configuration Options
+ url: sql-performance-tuning.html#other-configuration-options
+ - text: Broadcast Hint for SQL Queries
+ url: sql-performance-tuning.html#broadcast-hint-for-sql-queries
+- text: Distributed SQL Engine
+ url: sql-distributed-sql-engine.html
+ subitems:
+ - text: "Running the Thrift JDBC/ODBC server"
+ url: sql-distributed-sql-engine.html#running-the-thrift-jdbcodbc-server
+ - text: Running the Spark SQL CLI
+ url: sql-distributed-sql-engine.html#running-the-spark-sql-cli
+- text: PySpark Usage Guide for Pandas with Apache Arrow
+ url: sql-pyspark-pandas-with-arrow.html
+ subitems:
+ - text: Apache Arrow in Spark
+ url: sql-pyspark-pandas-with-arrow.html#apache-arrow-in-spark
+ - text: "Enabling for Conversion to/from Pandas"
+ url: sql-pyspark-pandas-with-arrow.html#enabling-for-conversion-tofrom-pandas
+ - text: "Pandas UDFs (a.k.a. Vectorized UDFs)"
+ url: sql-pyspark-pandas-with-arrow.html#pandas-udfs-aka-vectorized-udfs
+ - text: Usage Notes
+ url: sql-pyspark-pandas-with-arrow.html#usage-notes
+- text: Migration Guide
+ url: sql-migration-guide.html
+ subitems:
+ - text: Spark SQL Upgrading Guide
+ url: sql-migration-guide-upgrade.html
+ - text: Compatibility with Apache Hive
+ url: sql-migration-guide-hive-compatibility.html
+- text: Reference
+ url: sql-reference.html
+ subitems:
+ - text: Data Types
+ url: sql-reference.html#data-types
+ - text: NaN Semantics
+ url: sql-reference.html#nan-semantics
+ - text: Arithmetic operations
+ url: sql-reference.html#arithmetic-operations
diff --git a/docs/_includes/nav-left-wrapper-sql.html b/docs/_includes/nav-left-wrapper-sql.html
new file mode 100644
index 0000000000000..edc4cf4514d01
--- /dev/null
+++ b/docs/_includes/nav-left-wrapper-sql.html
@@ -0,0 +1,6 @@
+
\ No newline at end of file
diff --git a/docs/_includes/nav-left.html b/docs/_includes/nav-left.html
index 73176f4132554..19d68fd191635 100644
--- a/docs/_includes/nav-left.html
+++ b/docs/_includes/nav-left.html
@@ -10,7 +10,8 @@
{% endif %}
- {% if item.subitems and navurl contains item.url %}
+ {% assign tag = item.url | remove: ".html" %}
+ {% if item.subitems and navurl contains tag %}
{% include nav-left.html nav=item.subitems %}
{% endif %}
{% endfor %}
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 88d549c3f1010..63e9a42775624 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -126,8 +126,12 @@
- {% if page.url contains "/ml" %}
- {% include nav-left-wrapper-ml.html nav-mllib=site.data.menu-mllib nav-ml=site.data.menu-ml %}
+ {% if page.url contains "/ml" or page.url contains "/sql" %}
+ {% if page.url contains "/ml" %}
+ {% include nav-left-wrapper-ml.html nav-mllib=site.data.menu-mllib nav-ml=site.data.menu-ml %}
+ {% else %}
+ {% include nav-left-wrapper-sql.html nav-sql=site.data.menu-sql %}
+ {% endif %}
-
+
@@ -184,7 +188,8 @@
{{ page.title }}
});
};
script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') +
- 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML';
+ 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' +
+ '?config=TeX-AMS-MML_HTMLorMML';
d.getElementsByTagName('head')[0].appendChild(script);
}(document));
diff --git a/docs/building-spark.md b/docs/building-spark.md
index 1d3e0b1b7d396..810c944333ee9 100644
--- a/docs/building-spark.md
+++ b/docs/building-spark.md
@@ -12,14 +12,14 @@ redirect_from: "building-with-maven.html"
## Apache Maven
The Maven-based build is the build of reference for Apache Spark.
-Building Spark using Maven requires Maven 3.3.9 or newer and Java 8+.
+Building Spark using Maven requires Maven 3.5.4 and Java 8.
Note that support for Java 7 was removed as of Spark 2.2.0.
### Setting up Maven's Memory Usage
You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`:
- export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m"
+ export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g"
(The `ReservedCodeCacheSize` setting is optional but recommended.)
If you don't add these parameters to `MAVEN_OPTS`, you may see errors and warnings like the following:
@@ -67,7 +67,7 @@ Examples:
./build/mvn -Pyarn -DskipTests clean package
# Apache Hadoop 2.7.X and later
- ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.7 -DskipTests clean package
+ ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.3 -DskipTests clean package
## Building With Hive and JDBC Support
@@ -236,8 +236,8 @@ The run-tests script also can be limited to a specific Python version or a speci
To run the SparkR tests you will need to install the [knitr](https://cran.r-project.org/package=knitr), [rmarkdown](https://cran.r-project.org/package=rmarkdown), [testthat](https://cran.r-project.org/package=testthat), [e1071](https://cran.r-project.org/package=e1071) and [survival](https://cran.r-project.org/package=survival) packages first:
- R -e "install.packages(c('knitr', 'rmarkdown', 'devtools', 'e1071', 'survival'), repos='http://cran.us.r-project.org')"
- R -e "devtools::install_version('testthat', version = '1.0.2', repos='http://cran.us.r-project.org')"
+ Rscript -e "install.packages(c('knitr', 'rmarkdown', 'devtools', 'e1071', 'survival'), repos='https://cloud.r-project.org/')"
+ Rscript -e "devtools::install_version('testthat', version = '1.0.2', repos='https://cloud.r-project.org/')"
You can run just the SparkR tests using the command:
diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md
index 36753f6373b55..b455cd4ba95df 100644
--- a/docs/cloud-integration.md
+++ b/docs/cloud-integration.md
@@ -87,6 +87,7 @@ is set to the chosen version of Spark:
org.apache.spark
hadoop-cloud_2.11
${spark.version}
+
provided
...
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index 7277e2fb2731d..1f0822f7a317b 100644
--- a/docs/cluster-overview.md
+++ b/docs/cluster-overview.md
@@ -45,7 +45,7 @@ There are several useful things to note about this architecture:
# Cluster Manager Types
-The system currently supports three cluster managers:
+The system currently supports several cluster managers:
* [Standalone](spark-standalone.html) -- a simple cluster manager included with Spark that makes it
easy to set up a cluster.
diff --git a/docs/configuration.md b/docs/configuration.md
index f344bcd20087d..4db0ea65518ed 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -188,8 +188,10 @@ of the most common options to set are:
unless otherwise specified. If set, PySpark memory for an executor will be
limited to this amount. If not set, Spark will not limit Python's memory use
and it is up to the application to avoid exceeding the overhead memory space
- shared with other non-JVM processes. When PySpark is run in YARN, this memory
+ shared with other non-JVM processes. When PySpark is run in YARN or Kubernetes, this memory
is added to executor resource requests.
+
+ NOTE: Python memory usage may not be limited on platforms that do not support resource limiting, such as Windows.
spark.blacklist.enabled
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 35293348e3f3d..cb96fd773aa5a 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -726,7 +726,7 @@ class GraphOps[VD, ED] {
var g = mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache()
// compute the messages
- var messages = g.mapReduceTriplets(sendMsg, mergeMsg)
+ var messages = GraphXUtils.mapReduceTriplets(g, sendMsg, mergeMsg)
var activeMessages = messages.count()
// Loop until no messages remain or maxIterations is achieved
var i = 0
diff --git a/docs/index.md b/docs/index.md
index 40f628b794c01..9a59097bb4efb 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -10,6 +10,11 @@ It provides high-level APIs in Java, Scala, Python and R,
and an optimized engine that supports general execution graphs.
It also supports a rich set of higher-level tools including [Spark SQL](sql-programming-guide.html) for SQL and structured data processing, [MLlib](ml-guide.html) for machine learning, [GraphX](graphx-programming-guide.html) for graph processing, and [Spark Streaming](streaming-programming-guide.html).
+# Security
+
+Security in Spark is OFF by default. This could mean you are vulnerable to attack by default.
+Please see [Spark Security](security.html) before downloading and running Spark.
+
# Downloading
Get Spark from the [downloads page](https://spark.apache.org/downloads.html) of the project website. This documentation is for Spark version {{site.SPARK_VERSION}}. Spark uses Hadoop's client libraries for HDFS and YARN. Downloads are pre-packaged for a handful of popular Hadoop versions.
@@ -26,12 +31,13 @@ Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). It's easy
locally on one machine --- all you need is to have `java` installed on your system `PATH`,
or the `JAVA_HOME` environment variable pointing to a Java installation.
-Spark runs on Java 8+, Python 2.7+/3.4+ and R 3.1+. For the Scala API, Spark {{site.SPARK_VERSION}}
+Spark runs on Java 8, Python 2.7+/3.4+ and R 3.1+. For the Scala API, Spark {{site.SPARK_VERSION}}
uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version
({{site.SCALA_BINARY_VERSION}}.x).
Note that support for Java 7, Python 2.6 and old Hadoop versions before 2.6.5 were removed as of Spark 2.2.0.
-Support for Scala 2.10 was removed as of 2.3.0.
+Support for Scala 2.10 was removed as of 2.3.0. Support for Scala 2.11 is deprecated as of Spark 2.4.1
+and will be removed in Spark 3.0.
# Running the Examples and Shell
diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js
index 13514e11b9332..287fce61540e9 100644
--- a/docs/js/api-docs.js
+++ b/docs/js/api-docs.js
@@ -58,6 +58,7 @@ $(document).ready(function() {
});
};
script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') +
- 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML';
+ 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' +
+ '?config=TeX-AMS-MML_HTMLorMML';
document.getElementsByTagName('head')[0].appendChild(script);
});
diff --git a/docs/js/vendor/jquery-1.12.4.min.js b/docs/js/vendor/jquery-1.12.4.min.js
new file mode 100755
index 0000000000000..e836475870da6
--- /dev/null
+++ b/docs/js/vendor/jquery-1.12.4.min.js
@@ -0,0 +1,5 @@
+/*! jQuery v1.12.4 | (c) jQuery Foundation | jquery.org/license */
+!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=a.document,e=c.slice,f=c.concat,g=c.push,h=c.indexOf,i={},j=i.toString,k=i.hasOwnProperty,l={},m="1.12.4",n=function(a,b){return new n.fn.init(a,b)},o=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,p=/^-ms-/,q=/-([\da-z])/gi,r=function(a,b){return b.toUpperCase()};n.fn=n.prototype={jquery:m,constructor:n,selector:"",length:0,toArray:function(){return e.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:e.call(this)},pushStack:function(a){var b=n.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a){return n.each(this,a)},map:function(a){return this.pushStack(n.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(e.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor()},push:g,sort:c.sort,splice:c.splice},n.extend=n.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||n.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(n.isPlainObject(c)||(b=n.isArray(c)))?(b?(b=!1,f=a&&n.isArray(a)?a:[]):f=a&&n.isPlainObject(a)?a:{},g[d]=n.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},n.extend({expando:"jQuery"+(m+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===n.type(a)},isArray:Array.isArray||function(a){return"array"===n.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){var b=a&&a.toString();return!n.isArray(a)&&b-parseFloat(b)+1>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==n.type(a)||a.nodeType||n.isWindow(a))return!1;try{if(a.constructor&&!k.call(a,"constructor")&&!k.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(!l.ownFirst)for(b in a)return k.call(a,b);for(b in a);return void 0===b||k.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?i[j.call(a)]||"object":typeof a},globalEval:function(b){b&&n.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(p,"ms-").replace(q,r)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b){var c,d=0;if(s(a)){for(c=a.length;c>d;d++)if(b.call(a[d],d,a[d])===!1)break}else for(d in a)if(b.call(a[d],d,a[d])===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(o,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(s(Object(a))?n.merge(c,"string"==typeof a?[a]:a):g.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(h)return h.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,e,g=0,h=[];if(s(a))for(d=a.length;d>g;g++)e=b(a[g],g,c),null!=e&&h.push(e);else for(g in a)e=b(a[g],g,c),null!=e&&h.push(e);return f.apply([],h)},guid:1,proxy:function(a,b){var c,d,f;return"string"==typeof b&&(f=a[b],b=a,a=f),n.isFunction(a)?(c=e.call(arguments,2),d=function(){return a.apply(b||this,c.concat(e.call(arguments)))},d.guid=a.guid=a.guid||n.guid++,d):void 0},now:function(){return+new Date},support:l}),"function"==typeof Symbol&&(n.fn[Symbol.iterator]=c[Symbol.iterator]),n.each("Boolean Number String Function Array Date RegExp Object Error Symbol".split(" "),function(a,b){i["[object "+b+"]"]=b.toLowerCase()});function s(a){var b=!!a&&"length"in a&&a.length,c=n.type(a);return"function"===c||n.isWindow(a)?!1:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var t=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+1*new Date,v=a.document,w=0,x=0,y=ga(),z=ga(),A=ga(),B=function(a,b){return a===b&&(l=!0),0},C=1<<31,D={}.hasOwnProperty,E=[],F=E.pop,G=E.push,H=E.push,I=E.slice,J=function(a,b){for(var c=0,d=a.length;d>c;c++)if(a[c]===b)return c;return-1},K="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",L="[\\x20\\t\\r\\n\\f]",M="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",N="\\["+L+"*("+M+")(?:"+L+"*([*^$|!~]?=)"+L+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+M+"))|)"+L+"*\\]",O=":("+M+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+N+")*)|.*)\\)|)",P=new RegExp(L+"+","g"),Q=new RegExp("^"+L+"+|((?:^|[^\\\\])(?:\\\\.)*)"+L+"+$","g"),R=new RegExp("^"+L+"*,"+L+"*"),S=new RegExp("^"+L+"*([>+~]|"+L+")"+L+"*"),T=new RegExp("="+L+"*([^\\]'\"]*?)"+L+"*\\]","g"),U=new RegExp(O),V=new RegExp("^"+M+"$"),W={ID:new RegExp("^#("+M+")"),CLASS:new RegExp("^\\.("+M+")"),TAG:new RegExp("^("+M+"|[*])"),ATTR:new RegExp("^"+N),PSEUDO:new RegExp("^"+O),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+L+"*(even|odd|(([+-]|)(\\d*)n|)"+L+"*(?:([+-]|)"+L+"*(\\d+)|))"+L+"*\\)|)","i"),bool:new RegExp("^(?:"+K+")$","i"),needsContext:new RegExp("^"+L+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+L+"*((?:-\\d)?\\d*)"+L+"*\\)|)(?=[^-]|$)","i")},X=/^(?:input|select|textarea|button)$/i,Y=/^h\d$/i,Z=/^[^{]+\{\s*\[native \w/,$=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,_=/[+~]/,aa=/'|\\/g,ba=new RegExp("\\\\([\\da-f]{1,6}"+L+"?|("+L+")|.)","ig"),ca=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)},da=function(){m()};try{H.apply(E=I.call(v.childNodes),v.childNodes),E[v.childNodes.length].nodeType}catch(ea){H={apply:E.length?function(a,b){G.apply(a,I.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function fa(a,b,d,e){var f,h,j,k,l,o,r,s,w=b&&b.ownerDocument,x=b?b.nodeType:9;if(d=d||[],"string"!=typeof a||!a||1!==x&&9!==x&&11!==x)return d;if(!e&&((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,p)){if(11!==x&&(o=$.exec(a)))if(f=o[1]){if(9===x){if(!(j=b.getElementById(f)))return d;if(j.id===f)return d.push(j),d}else if(w&&(j=w.getElementById(f))&&t(b,j)&&j.id===f)return d.push(j),d}else{if(o[2])return H.apply(d,b.getElementsByTagName(a)),d;if((f=o[3])&&c.getElementsByClassName&&b.getElementsByClassName)return H.apply(d,b.getElementsByClassName(f)),d}if(c.qsa&&!A[a+" "]&&(!q||!q.test(a))){if(1!==x)w=b,s=a;else if("object"!==b.nodeName.toLowerCase()){(k=b.getAttribute("id"))?k=k.replace(aa,"\\$&"):b.setAttribute("id",k=u),r=g(a),h=r.length,l=V.test(k)?"#"+k:"[id='"+k+"']";while(h--)r[h]=l+" "+qa(r[h]);s=r.join(","),w=_.test(a)&&oa(b.parentNode)||b}if(s)try{return H.apply(d,w.querySelectorAll(s)),d}catch(y){}finally{k===u&&b.removeAttribute("id")}}}return i(a.replace(Q,"$1"),b,d,e)}function ga(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function ha(a){return a[u]=!0,a}function ia(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function ja(a,b){var c=a.split("|"),e=c.length;while(e--)d.attrHandle[c[e]]=b}function ka(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||C)-(~a.sourceIndex||C);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function la(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function ma(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function na(a){return ha(function(b){return b=+b,ha(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function oa(a){return a&&"undefined"!=typeof a.getElementsByTagName&&a}c=fa.support={},f=fa.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=fa.setDocument=function(a){var b,e,g=a?a.ownerDocument||a:v;return g!==n&&9===g.nodeType&&g.documentElement?(n=g,o=n.documentElement,p=!f(n),(e=n.defaultView)&&e.top!==e&&(e.addEventListener?e.addEventListener("unload",da,!1):e.attachEvent&&e.attachEvent("onunload",da)),c.attributes=ia(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ia(function(a){return a.appendChild(n.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=Z.test(n.getElementsByClassName),c.getById=ia(function(a){return o.appendChild(a).id=u,!n.getElementsByName||!n.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c=b.getElementById(a);return c?[c]:[]}},d.filter.ID=function(a){var b=a.replace(ba,ca);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(ba,ca);return function(a){var c="undefined"!=typeof a.getAttributeNode&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return"undefined"!=typeof b.getElementsByTagName?b.getElementsByTagName(a):c.qsa?b.querySelectorAll(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return"undefined"!=typeof b.getElementsByClassName&&p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=Z.test(n.querySelectorAll))&&(ia(function(a){o.appendChild(a).innerHTML=" ",a.querySelectorAll("[msallowcapture^='']").length&&q.push("[*^$]="+L+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+L+"*(?:value|"+K+")"),a.querySelectorAll("[id~="+u+"-]").length||q.push("~="),a.querySelectorAll(":checked").length||q.push(":checked"),a.querySelectorAll("a#"+u+"+*").length||q.push(".#.+[+~]")}),ia(function(a){var b=n.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+L+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=Z.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ia(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",O)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=Z.test(o.compareDocumentPosition),t=b||Z.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===n||a.ownerDocument===v&&t(v,a)?-1:b===n||b.ownerDocument===v&&t(v,b)?1:k?J(k,a)-J(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,e=a.parentNode,f=b.parentNode,g=[a],h=[b];if(!e||!f)return a===n?-1:b===n?1:e?-1:f?1:k?J(k,a)-J(k,b):0;if(e===f)return ka(a,b);c=a;while(c=c.parentNode)g.unshift(c);c=b;while(c=c.parentNode)h.unshift(c);while(g[d]===h[d])d++;return d?ka(g[d],h[d]):g[d]===v?-1:h[d]===v?1:0},n):n},fa.matches=function(a,b){return fa(a,null,null,b)},fa.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(T,"='$1']"),c.matchesSelector&&p&&!A[b+" "]&&(!r||!r.test(b))&&(!q||!q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return fa(b,n,null,[a]).length>0},fa.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},fa.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&D.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},fa.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},fa.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=fa.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=fa.selectors={cacheLength:50,createPseudo:ha,match:W,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(ba,ca),a[3]=(a[3]||a[4]||a[5]||"").replace(ba,ca),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||fa.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&fa.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return W.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&U.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(ba,ca).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+L+")"+a+"("+L+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||"undefined"!=typeof a.getAttribute&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=fa.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e.replace(P," ")+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h,t=!1;if(q){if(f){while(p){m=b;while(m=m[p])if(h?m.nodeName.toLowerCase()===r:1===m.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){m=q,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n&&j[2],m=n&&q.childNodes[n];while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if(1===m.nodeType&&++t&&m===b){k[a]=[w,n,t];break}}else if(s&&(m=b,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n),t===!1)while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if((h?m.nodeName.toLowerCase()===r:1===m.nodeType)&&++t&&(s&&(l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),k[a]=[w,t]),m===b))break;return t-=e,t===d||t%d===0&&t/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||fa.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?ha(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=J(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:ha(function(a){var b=[],c=[],d=h(a.replace(Q,"$1"));return d[u]?ha(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),b[0]=null,!c.pop()}}),has:ha(function(a){return function(b){return fa(a,b).length>0}}),contains:ha(function(a){return a=a.replace(ba,ca),function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:ha(function(a){return V.test(a||"")||fa.error("unsupported lang: "+a),a=a.replace(ba,ca).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Y.test(a.nodeName)},input:function(a){return X.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:na(function(){return[0]}),last:na(function(a,b){return[b-1]}),eq:na(function(a,b,c){return[0>c?c+b:c]}),even:na(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:na(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:na(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:na(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function ra(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j,k=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(j=b[u]||(b[u]={}),i=j[b.uniqueID]||(j[b.uniqueID]={}),(h=i[d])&&h[0]===w&&h[1]===f)return k[2]=h[2];if(i[d]=k,k[2]=a(b,c,g))return!0}}}function sa(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function ta(a,b,c){for(var d=0,e=b.length;e>d;d++)fa(a,b[d],c);return c}function ua(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(c&&!c(f,d,e)||(g.push(f),j&&b.push(h)));return g}function va(a,b,c,d,e,f){return d&&!d[u]&&(d=va(d)),e&&!e[u]&&(e=va(e,f)),ha(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||ta(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:ua(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=ua(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?J(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=ua(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):H.apply(g,r)})}function wa(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=ra(function(a){return a===b},h,!0),l=ra(function(a){return J(b,a)>-1},h,!0),m=[function(a,c,d){var e=!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d));return b=null,e}];f>i;i++)if(c=d.relative[a[i].type])m=[ra(sa(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return va(i>1&&sa(m),i>1&&qa(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(Q,"$1"),c,e>i&&wa(a.slice(i,e)),f>e&&wa(a=a.slice(e)),f>e&&qa(a))}m.push(c)}return sa(m)}function xa(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,o,q,r=0,s="0",t=f&&[],u=[],v=j,x=f||e&&d.find.TAG("*",k),y=w+=null==v?1:Math.random()||.1,z=x.length;for(k&&(j=g===n||g||k);s!==z&&null!=(l=x[s]);s++){if(e&&l){o=0,g||l.ownerDocument===n||(m(l),h=!p);while(q=a[o++])if(q(l,g||n,h)){i.push(l);break}k&&(w=y)}c&&((l=!q&&l)&&r--,f&&t.push(l))}if(r+=s,c&&s!==r){o=0;while(q=b[o++])q(t,u,g,h);if(f){if(r>0)while(s--)t[s]||u[s]||(u[s]=F.call(i));u=ua(u)}H.apply(i,u),k&&!f&&u.length>0&&r+b.length>1&&fa.uniqueSort(i)}return k&&(w=y,j=v),t};return c?ha(f):f}return h=fa.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=wa(b[c]),f[u]?d.push(f):e.push(f);f=A(a,xa(e,d)),f.selector=a}return f},i=fa.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(ba,ca),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=W.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(ba,ca),_.test(j[0].type)&&oa(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&qa(j),!a)return H.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,!b||_.test(a)&&oa(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ia(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),ia(function(a){return a.innerHTML=" ","#"===a.firstChild.getAttribute("href")})||ja("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ia(function(a){return a.innerHTML=" ",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||ja("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),ia(function(a){return null==a.getAttribute("disabled")})||ja(K,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),fa}(a);n.find=t,n.expr=t.selectors,n.expr[":"]=n.expr.pseudos,n.uniqueSort=n.unique=t.uniqueSort,n.text=t.getText,n.isXMLDoc=t.isXML,n.contains=t.contains;var u=function(a,b,c){var d=[],e=void 0!==c;while((a=a[b])&&9!==a.nodeType)if(1===a.nodeType){if(e&&n(a).is(c))break;d.push(a)}return d},v=function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c},w=n.expr.match.needsContext,x=/^<([\w-]+)\s*\/?>(?:<\/\1>|)$/,y=/^.[^:#\[\.,]*$/;function z(a,b,c){if(n.isFunction(b))return n.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return n.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(y.test(b))return n.filter(b,a,c);b=n.filter(b,a)}return n.grep(a,function(a){return n.inArray(a,b)>-1!==c})}n.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?n.find.matchesSelector(d,a)?[d]:[]:n.find.matches(a,n.grep(b,function(a){return 1===a.nodeType}))},n.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(n(a).filter(function(){for(b=0;e>b;b++)if(n.contains(d[b],this))return!0}));for(b=0;e>b;b++)n.find(a,d[b],c);return c=this.pushStack(e>1?n.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(z(this,a||[],!1))},not:function(a){return this.pushStack(z(this,a||[],!0))},is:function(a){return!!z(this,"string"==typeof a&&w.test(a)?n(a):a||[],!1).length}});var A,B=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,C=n.fn.init=function(a,b,c){var e,f;if(!a)return this;if(c=c||A,"string"==typeof a){if(e="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:B.exec(a),!e||!e[1]&&b)return!b||b.jquery?(b||c).find(a):this.constructor(b).find(a);if(e[1]){if(b=b instanceof n?b[0]:b,n.merge(this,n.parseHTML(e[1],b&&b.nodeType?b.ownerDocument||b:d,!0)),x.test(e[1])&&n.isPlainObject(b))for(e in b)n.isFunction(this[e])?this[e](b[e]):this.attr(e,b[e]);return this}if(f=d.getElementById(e[2]),f&&f.parentNode){if(f.id!==e[2])return A.find(a);this.length=1,this[0]=f}return this.context=d,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):n.isFunction(a)?"undefined"!=typeof c.ready?c.ready(a):a(n):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),n.makeArray(a,this))};C.prototype=n.fn,A=n(d);var D=/^(?:parents|prev(?:Until|All))/,E={children:!0,contents:!0,next:!0,prev:!0};n.fn.extend({has:function(a){var b,c=n(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(n.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=w.test(a)||"string"!=typeof a?n(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&n.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?n.uniqueSort(f):f)},index:function(a){return a?"string"==typeof a?n.inArray(this[0],n(a)):n.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(n.uniqueSort(n.merge(this.get(),n(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function F(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}n.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return u(a,"parentNode")},parentsUntil:function(a,b,c){return u(a,"parentNode",c)},next:function(a){return F(a,"nextSibling")},prev:function(a){return F(a,"previousSibling")},nextAll:function(a){return u(a,"nextSibling")},prevAll:function(a){return u(a,"previousSibling")},nextUntil:function(a,b,c){return u(a,"nextSibling",c)},prevUntil:function(a,b,c){return u(a,"previousSibling",c)},siblings:function(a){return v((a.parentNode||{}).firstChild,a)},children:function(a){return v(a.firstChild)},contents:function(a){return n.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:n.merge([],a.childNodes)}},function(a,b){n.fn[a]=function(c,d){var e=n.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=n.filter(d,e)),this.length>1&&(E[a]||(e=n.uniqueSort(e)),D.test(a)&&(e=e.reverse())),this.pushStack(e)}});var G=/\S+/g;function H(a){var b={};return n.each(a.match(G)||[],function(a,c){b[c]=!0}),b}n.Callbacks=function(a){a="string"==typeof a?H(a):n.extend({},a);var b,c,d,e,f=[],g=[],h=-1,i=function(){for(e=a.once,d=b=!0;g.length;h=-1){c=g.shift();while(++h-1)f.splice(c,1),h>=c&&h--}),this},has:function(a){return a?n.inArray(a,f)>-1:f.length>0},empty:function(){return f&&(f=[]),this},disable:function(){return e=g=[],f=c="",this},disabled:function(){return!f},lock:function(){return e=!0,c||j.disable(),this},locked:function(){return!!e},fireWith:function(a,c){return e||(c=c||[],c=[a,c.slice?c.slice():c],g.push(c),b||i()),this},fire:function(){return j.fireWith(this,arguments),this},fired:function(){return!!d}};return j},n.extend({Deferred:function(a){var b=[["resolve","done",n.Callbacks("once memory"),"resolved"],["reject","fail",n.Callbacks("once memory"),"rejected"],["notify","progress",n.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return n.Deferred(function(c){n.each(b,function(b,f){var g=n.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&n.isFunction(a.promise)?a.promise().progress(c.notify).done(c.resolve).fail(c.reject):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?n.extend(a,d):d}},e={};return d.pipe=d.then,n.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=e.call(arguments),d=c.length,f=1!==d||a&&n.isFunction(a.promise)?d:0,g=1===f?a:n.Deferred(),h=function(a,b,c){return function(d){b[a]=this,c[a]=arguments.length>1?e.call(arguments):d,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(d>1)for(i=new Array(d),j=new Array(d),k=new Array(d);d>b;b++)c[b]&&n.isFunction(c[b].promise)?c[b].promise().progress(h(b,j,i)).done(h(b,k,c)).fail(g.reject):--f;return f||g.resolveWith(k,c),g.promise()}});var I;n.fn.ready=function(a){return n.ready.promise().done(a),this},n.extend({isReady:!1,readyWait:1,holdReady:function(a){a?n.readyWait++:n.ready(!0)},ready:function(a){(a===!0?--n.readyWait:n.isReady)||(n.isReady=!0,a!==!0&&--n.readyWait>0||(I.resolveWith(d,[n]),n.fn.triggerHandler&&(n(d).triggerHandler("ready"),n(d).off("ready"))))}});function J(){d.addEventListener?(d.removeEventListener("DOMContentLoaded",K),a.removeEventListener("load",K)):(d.detachEvent("onreadystatechange",K),a.detachEvent("onload",K))}function K(){(d.addEventListener||"load"===a.event.type||"complete"===d.readyState)&&(J(),n.ready())}n.ready.promise=function(b){if(!I)if(I=n.Deferred(),"complete"===d.readyState||"loading"!==d.readyState&&!d.documentElement.doScroll)a.setTimeout(n.ready);else if(d.addEventListener)d.addEventListener("DOMContentLoaded",K),a.addEventListener("load",K);else{d.attachEvent("onreadystatechange",K),a.attachEvent("onload",K);var c=!1;try{c=null==a.frameElement&&d.documentElement}catch(e){}c&&c.doScroll&&!function f(){if(!n.isReady){try{c.doScroll("left")}catch(b){return a.setTimeout(f,50)}J(),n.ready()}}()}return I.promise(b)},n.ready.promise();var L;for(L in n(l))break;l.ownFirst="0"===L,l.inlineBlockNeedsLayout=!1,n(function(){var a,b,c,e;c=d.getElementsByTagName("body")[0],c&&c.style&&(b=d.createElement("div"),e=d.createElement("div"),e.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(e).appendChild(b),"undefined"!=typeof b.style.zoom&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",l.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(e))}),function(){var a=d.createElement("div");l.deleteExpando=!0;try{delete a.test}catch(b){l.deleteExpando=!1}a=null}();var M=function(a){var b=n.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b},N=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,O=/([A-Z])/g;function P(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(O,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:N.test(c)?n.parseJSON(c):c}catch(e){}n.data(a,b,c)}else c=void 0;
+}return c}function Q(a){var b;for(b in a)if(("data"!==b||!n.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function R(a,b,d,e){if(M(a)){var f,g,h=n.expando,i=a.nodeType,j=i?n.cache:a,k=i?a[h]:a[h]&&h;if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||n.guid++:h),j[k]||(j[k]=i?{}:{toJSON:n.noop}),"object"!=typeof b&&"function"!=typeof b||(e?j[k]=n.extend(j[k],b):j[k].data=n.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[n.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[n.camelCase(b)])):f=g,f}}function S(a,b,c){if(M(a)){var d,e,f=a.nodeType,g=f?n.cache:a,h=f?a[n.expando]:n.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){n.isArray(b)?b=b.concat(n.map(b,n.camelCase)):b in d?b=[b]:(b=n.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!Q(d):!n.isEmptyObject(d))return}(c||(delete g[h].data,Q(g[h])))&&(f?n.cleanData([a],!0):l.deleteExpando||g!=g.window?delete g[h]:g[h]=void 0)}}}n.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?n.cache[a[n.expando]]:a[n.expando],!!a&&!Q(a)},data:function(a,b,c){return R(a,b,c)},removeData:function(a,b){return S(a,b)},_data:function(a,b,c){return R(a,b,c,!0)},_removeData:function(a,b){return S(a,b,!0)}}),n.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=n.data(f),1===f.nodeType&&!n._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=n.camelCase(d.slice(5)),P(f,d,e[d])));n._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){n.data(this,a)}):arguments.length>1?this.each(function(){n.data(this,a,b)}):f?P(f,a,n.data(f,a)):void 0},removeData:function(a){return this.each(function(){n.removeData(this,a)})}}),n.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=n._data(a,b),c&&(!d||n.isArray(c)?d=n._data(a,b,n.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=n.queue(a,b),d=c.length,e=c.shift(),f=n._queueHooks(a,b),g=function(){n.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return n._data(a,c)||n._data(a,c,{empty:n.Callbacks("once memory").add(function(){n._removeData(a,b+"queue"),n._removeData(a,c)})})}}),n.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},Z=/^(?:checkbox|radio)$/i,$=/<([\w:-]+)/,_=/^$|\/(?:java|ecma)script/i,aa=/^\s+/,ba="abbr|article|aside|audio|bdi|canvas|data|datalist|details|dialog|figcaption|figure|footer|header|hgroup|main|mark|meter|nav|output|picture|progress|section|summary|template|time|video";function ca(a){var b=ba.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return c}!function(){var a=d.createElement("div"),b=d.createDocumentFragment(),c=d.createElement("input");a.innerHTML=" a ",l.leadingWhitespace=3===a.firstChild.nodeType,l.tbody=!a.getElementsByTagName("tbody").length,l.htmlSerialize=!!a.getElementsByTagName("link").length,l.html5Clone="<:nav>"!==d.createElement("nav").cloneNode(!0).outerHTML,c.type="checkbox",c.checked=!0,b.appendChild(c),l.appendChecked=c.checked,a.innerHTML="",l.noCloneChecked=!!a.cloneNode(!0).lastChild.defaultValue,b.appendChild(a),c=d.createElement("input"),c.setAttribute("type","radio"),c.setAttribute("checked","checked"),c.setAttribute("name","t"),a.appendChild(c),l.checkClone=a.cloneNode(!0).cloneNode(!0).lastChild.checked,l.noCloneEvent=!!a.addEventListener,a[n.expando]=1,l.attributes=!a.getAttribute(n.expando)}();var da={option:[1,""," "],legend:[1,""," "],area:[1,""," "],param:[1,""," "],thead:[1,""],tr:[2,""],col:[2,""],td:[3,""],_default:l.htmlSerialize?[0,"",""]:[1,"X","
"]};da.optgroup=da.option,da.tbody=da.tfoot=da.colgroup=da.caption=da.thead,da.th=da.td;function ea(a,b){var c,d,e=0,f="undefined"!=typeof a.getElementsByTagName?a.getElementsByTagName(b||"*"):"undefined"!=typeof a.querySelectorAll?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||n.nodeName(d,b)?f.push(d):n.merge(f,ea(d,b));return void 0===b||b&&n.nodeName(a,b)?n.merge([a],f):f}function fa(a,b){for(var c,d=0;null!=(c=a[d]);d++)n._data(c,"globalEval",!b||n._data(b[d],"globalEval"))}var ga=/<|?\w+;/,ha=/r;r++)if(g=a[r],g||0===g)if("object"===n.type(g))n.merge(q,g.nodeType?[g]:g);else if(ga.test(g)){i=i||p.appendChild(b.createElement("div")),j=($.exec(g)||["",""])[1].toLowerCase(),m=da[j]||da._default,i.innerHTML=m[1]+n.htmlPrefilter(g)+m[2],f=m[0];while(f--)i=i.lastChild;if(!l.leadingWhitespace&&aa.test(g)&&q.push(b.createTextNode(aa.exec(g)[0])),!l.tbody){g="table"!==j||ha.test(g)?""!==m[1]||ha.test(g)?0:i:i.firstChild,f=g&&g.childNodes.length;while(f--)n.nodeName(k=g.childNodes[f],"tbody")&&!k.childNodes.length&&g.removeChild(k)}n.merge(q,i.childNodes),i.textContent="";while(i.firstChild)i.removeChild(i.firstChild);i=p.lastChild}else q.push(b.createTextNode(g));i&&p.removeChild(i),l.appendChecked||n.grep(ea(q,"input"),ia),r=0;while(g=q[r++])if(d&&n.inArray(g,d)>-1)e&&e.push(g);else if(h=n.contains(g.ownerDocument,g),i=ea(p.appendChild(g),"script"),h&&fa(i),c){f=0;while(g=i[f++])_.test(g.type||"")&&c.push(g)}return i=null,p}!function(){var b,c,e=d.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(l[b]=c in a)||(e.setAttribute(c,"t"),l[b]=e.attributes[c].expando===!1);e=null}();var ka=/^(?:input|select|textarea)$/i,la=/^key/,ma=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,na=/^(?:focusinfocus|focusoutblur)$/,oa=/^([^.]*)(?:\.(.+)|)/;function pa(){return!0}function qa(){return!1}function ra(){try{return d.activeElement}catch(a){}}function sa(a,b,c,d,e,f){var g,h;if("object"==typeof b){"string"!=typeof c&&(d=d||c,c=void 0);for(h in b)sa(a,h,c,d,b[h],f);return a}if(null==d&&null==e?(e=c,d=c=void 0):null==e&&("string"==typeof c?(e=d,d=void 0):(e=d,d=c,c=void 0)),e===!1)e=qa;else if(!e)return a;return 1===f&&(g=e,e=function(a){return n().off(a),g.apply(this,arguments)},e.guid=g.guid||(g.guid=n.guid++)),a.each(function(){n.event.add(this,b,e,d,c)})}n.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=n.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return"undefined"==typeof n||a&&n.event.triggered===a.type?void 0:n.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(G)||[""],h=b.length;while(h--)f=oa.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=n.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=n.event.special[o]||{},l=n.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&n.expr.match.needsContext.test(e),namespace:p.join(".")},i),(m=g[o])||(m=g[o]=[],m.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,l):m.push(l),n.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n.hasData(a)&&n._data(a);if(r&&(k=r.events)){b=(b||"").match(G)||[""],j=b.length;while(j--)if(h=oa.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=n.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,m=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=m.length;while(f--)g=m[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(m.splice(f,1),g.selector&&m.delegateCount--,l.remove&&l.remove.call(a,g));i&&!m.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||n.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)n.event.remove(a,o+b[j],c,d,!0);n.isEmptyObject(k)&&(delete r.handle,n._removeData(a,"events"))}},trigger:function(b,c,e,f){var g,h,i,j,l,m,o,p=[e||d],q=k.call(b,"type")?b.type:b,r=k.call(b,"namespace")?b.namespace.split("."):[];if(i=m=e=e||d,3!==e.nodeType&&8!==e.nodeType&&!na.test(q+n.event.triggered)&&(q.indexOf(".")>-1&&(r=q.split("."),q=r.shift(),r.sort()),h=q.indexOf(":")<0&&"on"+q,b=b[n.expando]?b:new n.Event(q,"object"==typeof b&&b),b.isTrigger=f?2:3,b.namespace=r.join("."),b.rnamespace=b.namespace?new RegExp("(^|\\.)"+r.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=e),c=null==c?[b]:n.makeArray(c,[b]),l=n.event.special[q]||{},f||!l.trigger||l.trigger.apply(e,c)!==!1)){if(!f&&!l.noBubble&&!n.isWindow(e)){for(j=l.delegateType||q,na.test(j+q)||(i=i.parentNode);i;i=i.parentNode)p.push(i),m=i;m===(e.ownerDocument||d)&&p.push(m.defaultView||m.parentWindow||a)}o=0;while((i=p[o++])&&!b.isPropagationStopped())b.type=o>1?j:l.bindType||q,g=(n._data(i,"events")||{})[b.type]&&n._data(i,"handle"),g&&g.apply(i,c),g=h&&i[h],g&&g.apply&&M(i)&&(b.result=g.apply(i,c),b.result===!1&&b.preventDefault());if(b.type=q,!f&&!b.isDefaultPrevented()&&(!l._default||l._default.apply(p.pop(),c)===!1)&&M(e)&&h&&e[q]&&!n.isWindow(e)){m=e[h],m&&(e[h]=null),n.event.triggered=q;try{e[q]()}catch(s){}n.event.triggered=void 0,m&&(e[h]=m)}return b.result}},dispatch:function(a){a=n.event.fix(a);var b,c,d,f,g,h=[],i=e.call(arguments),j=(n._data(this,"events")||{})[a.type]||[],k=n.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=n.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,c=0;while((g=f.handlers[c++])&&!a.isImmediatePropagationStopped())a.rnamespace&&!a.rnamespace.test(g.namespace)||(a.handleObj=g,a.data=g.data,d=((n.event.special[g.origType]||{}).handle||g.handler).apply(f.elem,i),void 0!==d&&(a.result=d)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&("click"!==a.type||isNaN(a.button)||a.button<1))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(d=[],c=0;h>c;c++)f=b[c],e=f.selector+" ",void 0===d[e]&&(d[e]=f.needsContext?n(e,this).index(i)>-1:n.find(e,this,null,[i]).length),d[e]&&d.push(f);d.length&&g.push({elem:i,handlers:d})}return h ]","i"),va=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:-]+)[^>]*)\/>/gi,wa=/