diff options
author | Josh Rosen <joshrosen@databricks.com> | 2016-04-14 16:43:28 -0700 |
---|---|---|
committer | Josh Rosen <joshrosen@databricks.com> | 2016-04-14 16:43:28 -0700 |
commit | ee4090b60e8b6a350913d1d5049f0770c251cd4a (patch) | |
tree | 7e082fa815430c23e0387461be0726cc3e4d04b5 /core | |
parent | 2407f5b14edcdcf750113766d82e78732f9852d6 (diff) | |
parent | d7e124edfe2578ecdf8e816a4dda3ce430a09172 (diff) | |
download | spark-ee4090b60e8b6a350913d1d5049f0770c251cd4a.tar.gz spark-ee4090b60e8b6a350913d1d5049f0770c251cd4a.tar.bz2 spark-ee4090b60e8b6a350913d1d5049f0770c251cd4a.zip |
Merge remote-tracking branch 'origin/master' into build-for-2.12
Diffstat (limited to 'core')
200 files changed, 3317 insertions, 1946 deletions
diff --git a/core/src/main/java/org/apache/spark/JavaSparkListener.java b/core/src/main/java/org/apache/spark/JavaSparkListener.java deleted file mode 100644 index 23bc9a2e81..0000000000 --- a/core/src/main/java/org/apache/spark/JavaSparkListener.java +++ /dev/null @@ -1,88 +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; - -import org.apache.spark.scheduler.*; - -/** - * Java clients should extend this class instead of implementing - * SparkListener directly. This is to prevent java clients - * from breaking when new events are added to the SparkListener - * trait. - * - * This is a concrete class instead of abstract to enforce - * new events get added to both the SparkListener and this adapter - * in lockstep. - */ -public class JavaSparkListener implements SparkListener { - - @Override - public void onStageCompleted(SparkListenerStageCompleted stageCompleted) { } - - @Override - public void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) { } - - @Override - public void onTaskStart(SparkListenerTaskStart taskStart) { } - - @Override - public void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) { } - - @Override - public void onTaskEnd(SparkListenerTaskEnd taskEnd) { } - - @Override - public void onJobStart(SparkListenerJobStart jobStart) { } - - @Override - public void onJobEnd(SparkListenerJobEnd jobEnd) { } - - @Override - public void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) { } - - @Override - public void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) { } - - @Override - public void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) { } - - @Override - public void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) { } - - @Override - public void onApplicationStart(SparkListenerApplicationStart applicationStart) { } - - @Override - public void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { } - - @Override - public void onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate executorMetricsUpdate) { } - - @Override - public void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { } - - @Override - public void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { } - - @Override - public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { } - - @Override - public void onOtherEvent(SparkListenerEvent event) { } - -} diff --git a/core/src/main/java/org/apache/spark/SparkExecutorInfo.java b/core/src/main/java/org/apache/spark/SparkExecutorInfo.java new file mode 100644 index 0000000000..dc3e826475 --- /dev/null +++ b/core/src/main/java/org/apache/spark/SparkExecutorInfo.java @@ -0,0 +1,33 @@ +/* + * 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 java.io.Serializable; + +/** + * Exposes information about Spark Executors. + * + * This interface is not designed to be implemented outside of Spark. We may add additional methods + * which may break binary compatibility with outside implementations. + */ +public interface SparkExecutorInfo extends Serializable { + String host(); + int port(); + long cacheSize(); + int numRunningTasks(); +} diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index e6b24afd88..97eed611e8 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -28,7 +28,7 @@ import org.apache.spark.scheduler.*; * this was a concrete Scala class, default implementations of new event handlers would be inherited * from the SparkListener trait). */ -public class SparkFirehoseListener implements SparkListener { +public class SparkFirehoseListener implements SparkListenerInterface { public void onEvent(SparkListenerEvent event) { } diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 666c797738..3fcb52f615 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -34,13 +34,13 @@ public class StorageLevels { public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, false, true, 2); public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, false, 1); public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, false, 2); - public static final StorageLevel OFF_HEAP = create(false, false, true, false, 1); + public static final StorageLevel OFF_HEAP = create(true, true, true, false, 1); /** * Create a new StorageLevel object. * @param useDisk saved to disk, if true - * @param useMemory saved to memory, if true - * @param useOffHeap saved to Tachyon, if true + * @param useMemory saved to on-heap memory, if true + * @param useOffHeap saved to off-heap memory, if true * @param deserialized saved as deserialized objects, if true * @param replication replication factor */ diff --git a/core/src/main/scala/org/apache/spark/io/LZ4BlockInputStream.java b/core/src/main/java/org/apache/spark/io/LZ4BlockInputStream.java index 27b6f0d4a3..8783b5f56e 100644 --- a/core/src/main/scala/org/apache/spark/io/LZ4BlockInputStream.java +++ b/core/src/main/java/org/apache/spark/io/LZ4BlockInputStream.java @@ -20,20 +20,17 @@ import java.io.IOException; import java.io.InputStream; import java.util.zip.Checksum; -import net.jpountz.lz4.LZ4BlockOutputStream; import net.jpountz.lz4.LZ4Exception; import net.jpountz.lz4.LZ4Factory; import net.jpountz.lz4.LZ4FastDecompressor; import net.jpountz.util.SafeUtils; -import net.jpountz.xxhash.StreamingXXHash32; -import net.jpountz.xxhash.XXHash32; import net.jpountz.xxhash.XXHashFactory; /** * {@link InputStream} implementation to decode data written with - * {@link LZ4BlockOutputStream}. This class is not thread-safe and does not + * {@link net.jpountz.lz4.LZ4BlockOutputStream}. This class is not thread-safe and does not * support {@link #mark(int)}/{@link #reset()}. - * @see LZ4BlockOutputStream + * @see net.jpountz.lz4.LZ4BlockOutputStream * * This is based on net.jpountz.lz4.LZ4BlockInputStream * @@ -90,12 +87,13 @@ public final class LZ4BlockInputStream extends FilterInputStream { } /** - * Create a new instance using {@link XXHash32} for checksuming. + * Create a new instance using {@link net.jpountz.xxhash.XXHash32} for checksuming. * @see #LZ4BlockInputStream(InputStream, LZ4FastDecompressor, Checksum) - * @see StreamingXXHash32#asChecksum() + * @see net.jpountz.xxhash.StreamingXXHash32#asChecksum() */ public LZ4BlockInputStream(InputStream in, LZ4FastDecompressor decompressor) { - this(in, decompressor, XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum()); + this(in, decompressor, + XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum()); } /** diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 81ee7ab58a..3c2980e442 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -215,8 +215,6 @@ final class ShuffleExternalSorter extends MemoryConsumer { } } - inMemSorter.reset(); - if (!isLastFile) { // i.e. this is a spill file // The current semantics of `shuffleRecordsWritten` seem to be that it's updated when records // are written to disk, not when they enter the shuffle sorting code. DiskBlockObjectWriter @@ -255,6 +253,10 @@ final class ShuffleExternalSorter extends MemoryConsumer { writeSortedFile(false); final long spillSize = freeMemory(); + inMemSorter.reset(); + // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the + // records. Otherwise, if the task is over allocated memory, then without freeing the memory pages, + // we might not be able to get memory for the pointer array. taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); return spillSize; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index fe79ff0e30..76b0e6a304 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -51,9 +51,12 @@ final class ShuffleInMemorySorter { */ private int pos = 0; + private int initialSize; + ShuffleInMemorySorter(MemoryConsumer consumer, int initialSize) { this.consumer = consumer; assert (initialSize > 0); + this.initialSize = initialSize; this.array = consumer.allocateArray(initialSize); this.sorter = new Sorter<>(ShuffleSortDataFormat.INSTANCE); } @@ -70,6 +73,10 @@ final class ShuffleInMemorySorter { } public void reset() { + if (consumer != null) { + consumer.freeArray(array); + this.array = consumer.allocateArray(initialSize); + } pos = 0; } diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 9aacb084f6..6807710f9f 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -56,9 +56,10 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillWriter; * Bytes 4 to 8: len(k) * Bytes 8 to 8 + len(k): key data * Bytes 8 + len(k) to 8 + len(k) + len(v): value data + * Bytes 8 + len(k) + len(v) to 8 + len(k) + len(v) + 8: pointer to next pair * * This means that the first four bytes store the entire record (key + value) length. This format - * is consistent with {@link org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter}, + * is compatible with {@link org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter}, * so we can pass records from this map directly into the sorter to sort records in place. */ public final class BytesToBytesMap extends MemoryConsumer { @@ -132,7 +133,12 @@ public final class BytesToBytesMap extends MemoryConsumer { /** * Number of keys defined in the map. */ - private int numElements; + private int numKeys; + + /** + * Number of values defined in the map. A key could have multiple values. + */ + private int numValues; /** * The map will be expanded once the number of keys exceeds this threshold. @@ -223,7 +229,12 @@ public final class BytesToBytesMap extends MemoryConsumer { /** * Returns the number of keys defined in the map. */ - public int numElements() { return numElements; } + public int numKeys() { return numKeys; } + + /** + * Returns the number of values defined in the map. A key could have multiple values. + */ + public int numValues() { return numValues; } public final class MapIterator implements Iterator<Location> { @@ -311,7 +322,8 @@ public final class BytesToBytesMap extends MemoryConsumer { if (currentPage != null) { int totalLength = Platform.getInt(pageBaseObject, offsetInPage); loc.with(currentPage, offsetInPage); - offsetInPage += 4 + totalLength; + // [total size] [key size] [key] [value] [pointer to next] + offsetInPage += 4 + totalLength + 8; recordsInPage --; return loc; } else { @@ -361,7 +373,7 @@ public final class BytesToBytesMap extends MemoryConsumer { while (numRecords > 0) { int length = Platform.getInt(base, offset); writer.write(base, offset + 4, length, 0); - offset += 4 + length; + offset += 4 + length + 8; numRecords--; } writer.close(); @@ -395,7 +407,7 @@ public final class BytesToBytesMap extends MemoryConsumer { * `lookup()`, the behavior of the returned iterator is undefined. */ public MapIterator iterator() { - return new MapIterator(numElements, loc, false); + return new MapIterator(numValues, loc, false); } /** @@ -409,7 +421,7 @@ public final class BytesToBytesMap extends MemoryConsumer { * `lookup()`, the behavior of the returned iterator is undefined. */ public MapIterator destructiveIterator() { - return new MapIterator(numElements, loc, true); + return new MapIterator(numValues, loc, true); } /** @@ -560,6 +572,20 @@ public final class BytesToBytesMap extends MemoryConsumer { } /** + * Find the next pair that has the same key as current one. + */ + public boolean nextValue() { + assert isDefined; + long nextAddr = Platform.getLong(baseObject, valueOffset + valueLength); + if (nextAddr == 0) { + return false; + } else { + updateAddressesAndSizes(nextAddr); + return true; + } + } + + /** * Returns the memory page that contains the current record. * This is only valid if this is returned by {@link BytesToBytesMap#iterator()}. */ @@ -625,10 +651,9 @@ public final class BytesToBytesMap extends MemoryConsumer { } /** - * Store a new key and value. This method may only be called once for a given key; if you want - * to update the value associated with a key, then you can directly manipulate the bytes stored - * at the value address. The return value indicates whether the put succeeded or whether it - * failed because additional memory could not be acquired. + * Append a new value for the key. This method could be called multiple times for a given key. + * The return value indicates whether the put succeeded or whether it failed because additional + * memory could not be acquired. * <p> * It is only valid to call this method immediately after calling `lookup()` using the same key. * </p> @@ -637,7 +662,7 @@ public final class BytesToBytesMap extends MemoryConsumer { * </p> * <p> * After calling this method, calls to `get[Key|Value]Address()` and `get[Key|Value]Length` - * will return information on the data stored by this `putNewKey` call. + * will return information on the data stored by this `append` call. * </p> * <p> * As an example usage, here's the proper way to store a new key: @@ -645,7 +670,7 @@ public final class BytesToBytesMap extends MemoryConsumer { * <pre> * Location loc = map.lookup(keyBase, keyOffset, keyLength); * if (!loc.isDefined()) { - * if (!loc.putNewKey(keyBase, keyOffset, keyLength, ...)) { + * if (!loc.append(keyBase, keyOffset, keyLength, ...)) { * // handle failure to grow map (by spilling, for example) * } * } @@ -657,26 +682,23 @@ public final class BytesToBytesMap extends MemoryConsumer { * @return true if the put() was successful and false if the put() failed because memory could * not be acquired. */ - public boolean putNewKey(Object keyBase, long keyOffset, int keyLength, - Object valueBase, long valueOffset, int valueLength) { - assert (!isDefined) : "Can only set value once for a key"; - assert (keyLength % 8 == 0); - assert (valueLength % 8 == 0); - assert(longArray != null); + public boolean append(Object kbase, long koff, int klen, Object vbase, long voff, int vlen) { + assert (klen % 8 == 0); + assert (vlen % 8 == 0); + assert (longArray != null); - - if (numElements == MAX_CAPACITY + if (numKeys == MAX_CAPACITY // The map could be reused from last spill (because of no enough memory to grow), // then we don't try to grow again if hit the `growthThreshold`. - || !canGrowArray && numElements > growthThreshold) { + || !canGrowArray && numKeys > growthThreshold) { return false; } // Here, we'll copy the data into our data pages. Because we only store a relative offset from // the key address instead of storing the absolute address of the value, the key and value // must be stored in the same memory page. - // (8 byte key length) (key) (value) - final long recordLength = 8 + keyLength + valueLength; + // (8 byte key length) (key) (value) (8 byte pointer to next value) + final long recordLength = 8 + klen + vlen + 8; if (currentPage == null || currentPage.size() - pageCursor < recordLength) { if (!acquireNewPage(recordLength + 4L)) { return false; @@ -687,30 +709,36 @@ public final class BytesToBytesMap extends MemoryConsumer { final Object base = currentPage.getBaseObject(); long offset = currentPage.getBaseOffset() + pageCursor; final long recordOffset = offset; - Platform.putInt(base, offset, keyLength + valueLength + 4); - Platform.putInt(base, offset + 4, keyLength); + Platform.putInt(base, offset, klen + vlen + 4); + Platform.putInt(base, offset + 4, klen); offset += 8; - Platform.copyMemory(keyBase, keyOffset, base, offset, keyLength); - offset += keyLength; - Platform.copyMemory(valueBase, valueOffset, base, offset, valueLength); + Platform.copyMemory(kbase, koff, base, offset, klen); + offset += klen; + Platform.copyMemory(vbase, voff, base, offset, vlen); + offset += vlen; + // put this value at the beginning of the list + Platform.putLong(base, offset, isDefined ? longArray.get(pos * 2) : 0); // --- Update bookkeeping data structures ---------------------------------------------------- offset = currentPage.getBaseOffset(); Platform.putInt(base, offset, Platform.getInt(base, offset) + 1); pageCursor += recordLength; - numElements++; final long storedKeyAddress = taskMemoryManager.encodePageNumberAndOffset( currentPage, recordOffset); longArray.set(pos * 2, storedKeyAddress); - longArray.set(pos * 2 + 1, keyHashcode); updateAddressesAndSizes(storedKeyAddress); - isDefined = true; + numValues++; + if (!isDefined) { + numKeys++; + longArray.set(pos * 2 + 1, keyHashcode); + isDefined = true; - if (numElements > growthThreshold && longArray.size() < MAX_CAPACITY) { - try { - growAndRehash(); - } catch (OutOfMemoryError oom) { - canGrowArray = false; + if (numKeys > growthThreshold && longArray.size() < MAX_CAPACITY) { + try { + growAndRehash(); + } catch (OutOfMemoryError oom) { + canGrowArray = false; + } } } return true; @@ -866,7 +894,8 @@ public final class BytesToBytesMap extends MemoryConsumer { * Reset this map to initialized state. */ public void reset() { - numElements = 0; + numKeys = 0; + numValues = 0; longArray.zeroOut(); while (dataPages.size() > 0) { diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index cf7c9a299f..dc9a8db9c5 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -200,14 +200,17 @@ public final class UnsafeExternalSorter extends MemoryConsumer { spillWriter.write(baseObject, baseOffset, recordLength, sortedRecords.getKeyPrefix()); } spillWriter.close(); - - inMemSorter.reset(); } final long spillSize = freeMemory(); // Note that this is more-or-less going to be a multiple of the page size, so wasted space in // pages will currently be counted as memory spilled even though that space isn't actually // written to disk. This also counts the space needed to store the sorter's pointer array. + inMemSorter.reset(); + // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the + // records. Otherwise, if the task is over allocated memory, then without freeing the memory pages, + // we might not be able to get memory for the pointer array. + taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); return spillSize; diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 145c3a1950..01eae0e8dc 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -84,6 +84,8 @@ public final class UnsafeInMemorySorter { */ private int pos = 0; + private long initialSize; + public UnsafeInMemorySorter( final MemoryConsumer consumer, final TaskMemoryManager memoryManager, @@ -102,6 +104,7 @@ public final class UnsafeInMemorySorter { LongArray array) { this.consumer = consumer; this.memoryManager = memoryManager; + this.initialSize = array.size(); if (recordComparator != null) { this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE); this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager); @@ -123,6 +126,10 @@ public final class UnsafeInMemorySorter { } public void reset() { + if (consumer != null) { + consumer.freeArray(array); + this.array = consumer.allocateArray(initialSize); + } pos = 0; } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java index 2b1c860e55..01aed95878 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java @@ -45,7 +45,7 @@ final class UnsafeSorterSpillMerger { } } }; - priorityQueue = new PriorityQueue<UnsafeSorterIterator>(numSpills, comparator); + priorityQueue = new PriorityQueue<>(numSpills, comparator); } /** diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index 0750488e4a..89a7963a86 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -28,8 +28,8 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: log4j.logger.org.apache.spark.repl.Main=WARN # Settings to quiet third party logs that are too verbose -log4j.logger.org.spark-project.jetty=WARN -log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 48f86d1536..47dd9162a1 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -106,21 +106,22 @@ pre { line-height: 18px; padding: 6px; margin: 0; + word-break: break-word; border-radius: 3px; } .stage-details { - max-height: 100px; overflow-y: auto; margin: 0; + display: block; transition: max-height 0.25s ease-out, padding 0.25s ease-out; } .stage-details.collapsed { - max-height: 0; padding-top: 0; padding-bottom: 0; border: none; + display: none; } .description-input { @@ -143,14 +144,15 @@ pre { max-height: 300px; overflow-y: auto; margin: 0; + display: block; transition: max-height 0.25s ease-out, padding 0.25s ease-out; } .stacktrace-details.collapsed { - max-height: 0; padding-top: 0; padding-bottom: 0; border: none; + display: none; } span.expand-additional-metrics, span.expand-dag-viz { diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 8fc657c5eb..76692ccec8 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -278,9 +278,9 @@ private object ContextCleaner { * Listener class used for testing when any item has been cleaned by the Cleaner class. */ private[spark] trait CleanerListener { - def rddCleaned(rddId: Int) - def shuffleCleaned(shuffleId: Int) - def broadcastCleaned(broadcastId: Long) - def accumCleaned(accId: Long) - def checkpointCleaned(rddId: Long) + def rddCleaned(rddId: Int): Unit + def shuffleCleaned(shuffleId: Int): Unit + def broadcastCleaned(broadcastId: Long): Unit + def accumCleaned(accId: Long): Unit + def checkpointCleaned(rddId: Long): Unit } diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala index 842bfdbadc..8baddf45bf 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala @@ -23,6 +23,10 @@ package org.apache.spark */ private[spark] trait ExecutorAllocationClient { + + /** Get the list of currently active executors */ + private[spark] def getExecutorIds(): Seq[String] + /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 509f5082f1..882d2b21cf 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -41,7 +41,7 @@ trait FutureAction[T] extends Future[T] { /** * Cancels the execution of this action. */ - def cancel() + def cancel(): Unit /** * Blocks until this action completes. @@ -65,7 +65,7 @@ trait FutureAction[T] extends Future[T] { * When this action is completed, either through an exception, or a value, applies the provided * function. */ - def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext) + def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext): Unit /** * Returns whether the action has already been completed with a value or an exception. @@ -156,16 +156,16 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** - * Handle via which a "run" function passed to a [[ComplexFutureAction]] - * can submit jobs for execution. - */ + * Handle via which a "run" function passed to a [[ComplexFutureAction]] + * can submit jobs for execution. + */ @DeveloperApi trait JobSubmitter { /** - * Submit a job for execution and return a FutureAction holding the result. - * This is a wrapper around the same functionality provided by SparkContext - * to enable cancellation. - */ + * Submit a job for execution and return a FutureAction holding the result. + * This is a wrapper around the same functionality provided by SparkContext + * to enable cancellation. + */ def submitJob[T, U, R]( rdd: RDD[T], processPartition: Iterator[T] => U, diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index e8748dd80a..2bdbd3fae9 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -56,7 +56,7 @@ private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) * Lives in the driver to receive heartbeats from executors.. */ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) - extends ThreadSafeRpcEndpoint with SparkListener with Logging { + extends SparkListener with ThreadSafeRpcEndpoint with Logging { def this(sc: SparkContext) { this(sc, new SystemClock) @@ -220,6 +220,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) } } -object HeartbeatReceiver { + +private[spark] object HeartbeatReceiver { val ENDPOINT_NAME = "HeartbeatReceiver" } diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 9fad1f6786..982b6d6b61 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -25,6 +25,7 @@ import org.eclipse.jetty.server.Server import org.eclipse.jetty.server.bio.SocketConnector import org.eclipse.jetty.server.ssl.SslSocketConnector import org.eclipse.jetty.servlet.{DefaultServlet, ServletContextHandler, ServletHolder} +import org.eclipse.jetty.util.component.LifeCycle import org.eclipse.jetty.util.security.{Constraint, Password} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -155,6 +156,12 @@ private[spark] class HttpServer( throw new ServerStateException("Server is already stopped") } else { server.stop() + // Stop the ThreadPool if it supports stop() method (through LifeCycle). + // It is needed because stopping the Server won't stop the ThreadPool it uses. + val threadPool = server.getThreadPool + if (threadPool != null && threadPool.isInstanceOf[LifeCycle]) { + threadPool.asInstanceOf[LifeCycle].stop + } port = -1 server = null } diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala index 7aa9057858..0dd4ec656f 100644 --- a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala @@ -187,7 +187,7 @@ private[spark] object InternalAccumulator { * add to the same set of accumulators. We do this to report the distribution of accumulator * values across all tasks within each stage. */ - def create(sc: SparkContext): Seq[Accumulator[_]] = { + def createAll(sc: SparkContext): Seq[Accumulator[_]] = { val accums = createAll() accums.foreach { accum => Accumulators.register(accum) diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala index 30db6ccbf4..719905a2c9 100644 --- a/core/src/main/scala/org/apache/spark/SSLOptions.scala +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -132,34 +132,35 @@ private[spark] case class SSLOptions( private[spark] object SSLOptions extends Logging { - /** Resolves SSLOptions settings from a given Spark configuration object at a given namespace. - * - * The following settings are allowed: - * $ - `[ns].enabled` - `true` or `false`, to enable or disable SSL respectively - * $ - `[ns].keyStore` - a path to the key-store file; can be relative to the current directory - * $ - `[ns].keyStorePassword` - a password to the key-store file - * $ - `[ns].keyPassword` - a password to the private key - * $ - `[ns].keyStoreType` - the type of the key-store - * $ - `[ns].needClientAuth` - whether SSL needs client authentication - * $ - `[ns].trustStore` - a path to the trust-store file; can be relative to the current - * directory - * $ - `[ns].trustStorePassword` - a password to the trust-store file - * $ - `[ns].trustStoreType` - the type of trust-store - * $ - `[ns].protocol` - a protocol name supported by a particular Java version - * $ - `[ns].enabledAlgorithms` - a comma separated list of ciphers - * - * For a list of protocols and ciphers supported by particular Java versions, you may go to - * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle - * blog page]]. - * - * You can optionally specify the default configuration. If you do, for each setting which is - * missing in SparkConf, the corresponding setting is used from the default configuration. - * - * @param conf Spark configuration object where the settings are collected from - * @param ns the namespace name - * @param defaults the default configuration - * @return [[org.apache.spark.SSLOptions]] object - */ + /** + * Resolves SSLOptions settings from a given Spark configuration object at a given namespace. + * + * The following settings are allowed: + * $ - `[ns].enabled` - `true` or `false`, to enable or disable SSL respectively + * $ - `[ns].keyStore` - a path to the key-store file; can be relative to the current directory + * $ - `[ns].keyStorePassword` - a password to the key-store file + * $ - `[ns].keyPassword` - a password to the private key + * $ - `[ns].keyStoreType` - the type of the key-store + * $ - `[ns].needClientAuth` - whether SSL needs client authentication + * $ - `[ns].trustStore` - a path to the trust-store file; can be relative to the current + * directory + * $ - `[ns].trustStorePassword` - a password to the trust-store file + * $ - `[ns].trustStoreType` - the type of trust-store + * $ - `[ns].protocol` - a protocol name supported by a particular Java version + * $ - `[ns].enabledAlgorithms` - a comma separated list of ciphers + * + * For a list of protocols and ciphers supported by particular Java versions, you may go to + * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle + * blog page]]. + * + * You can optionally specify the default configuration. If you do, for each setting which is + * missing in SparkConf, the corresponding setting is used from the default configuration. + * + * @param conf Spark configuration object where the settings are collected from + * @param ns the namespace name + * @param defaults the default configuration + * @return [[org.apache.spark.SSLOptions]] object + */ def parse(conf: SparkConf, ns: String, defaults: Option[SSLOptions] = None): SSLOptions = { val enabled = conf.getBoolean(s"$ns.enabled", defaultValue = defaults.exists(_.enabled)) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 5da2e98f1f..acce6bc24f 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -419,8 +419,10 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { */ private[spark] def getenv(name: String): String = System.getenv(name) - /** Checks for illegal or deprecated config settings. Throws an exception for the former. Not - * idempotent - may mutate this conf object to convert deprecated settings to supported ones. */ + /** + * Checks for illegal or deprecated config settings. Throws an exception for the former. Not + * idempotent - may mutate this conf object to convert deprecated settings to supported ones. + */ private[spark] def validateSettings() { if (contains("spark.local.dir")) { val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " + @@ -454,9 +456,9 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { "Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit." throw new Exception(msg) } - if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { - val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). " + - "Use spark.executor.memory instead." + if (javaOpts.contains("-Xmx")) { + val msg = s"$executorOptsKey is not allowed to specify max heap memory settings " + + s"(was '$javaOpts'). Use spark.executor.memory instead." throw new Exception(msg) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index dcb41f3a40..e41088f7c8 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -147,8 +147,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli appName: String, sparkHome: String = null, jars: Seq[String] = Nil, - environment: Map[String, String] = Map()) = - { + environment: Map[String, String] = Map()) = { this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment)) } @@ -603,8 +602,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** - * Set a local property that affects jobs submitted from this thread, such as the - * Spark fair scheduler pool. + * Set a local property that affects jobs submitted from this thread, such as the Spark fair + * scheduler pool. User-defined properties may also be set here. These properties are propagated + * through to worker tasks and can be accessed there via + * [[org.apache.spark.TaskContext#getLocalProperty]]. */ def setLocalProperty(key: String, value: String) { if (value == null) { @@ -722,7 +723,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli (safeEnd - safeStart) / step + 1 } } - parallelize(0 until numSlices, numSlices).mapPartitionsWithIndex((i, _) => { + parallelize(0 until numSlices, numSlices).mapPartitionsWithIndex { (i, _) => val partitionStart = (i * numElements) / numSlices * step + start val partitionEnd = (((i + 1) * numElements) / numSlices) * step + start def getSafeMargin(bi: BigInt): Long = @@ -761,7 +762,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli ret } } - }) + } } /** Distribute a local Scala collection to form an RDD. @@ -774,9 +775,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli parallelize(seq, numSlices) } - /** Distribute a local Scala collection to form an RDD, with one or more - * location preferences (hostnames of Spark nodes) for each object. - * Create a new partition for each collection item. */ + /** + * Distribute a local Scala collection to form an RDD, with one or more + * location preferences (hostnames of Spark nodes) for each object. + * Create a new partition for each collection item. + */ def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = withScope { assertNotStopped() val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap @@ -1096,14 +1099,15 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli new NewHadoopRDD(this, fClass, kClass, vClass, jconf) } - /** Get an RDD for a Hadoop SequenceFile with given key and value types. - * - * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle - * operation will create many references to the same object. - * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first - * copy them using a `map` function. - */ + /** + * Get an RDD for a Hadoop SequenceFile with given key and value types. + * + * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. + */ def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V], @@ -1114,14 +1118,15 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions) } - /** Get an RDD for a Hadoop SequenceFile with given key and value types. - * - * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle - * operation will create many references to the same object. - * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first - * copy them using a `map` function. - * */ + /** + * Get an RDD for a Hadoop SequenceFile with given key and value types. + * + * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. + */ def sequenceFile[K, V]( path: String, keyClass: Class[K], @@ -1353,10 +1358,20 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Register a listener to receive up-calls from events that happen during execution. */ @DeveloperApi - def addSparkListener(listener: SparkListener) { + def addSparkListener(listener: SparkListenerInterface) { listenerBus.addListener(listener) } + private[spark] override def getExecutorIds(): Seq[String] = { + schedulerBackend match { + case b: CoarseGrainedSchedulerBackend => + b.getExecutorIds() + case _ => + logWarning("Requesting executors is only supported in coarse-grained mode") + Nil + } + } + /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. @@ -1994,7 +2009,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Use reflection to find the right constructor val constructors = { val listenerClass = Utils.classForName(className) - listenerClass.getConstructors.asInstanceOf[Array[Constructor[_ <: SparkListener]]] + listenerClass + .getConstructors + .asInstanceOf[Array[Constructor[_ <: SparkListenerInterface]]] } val constructorTakingSparkConf = constructors.find { c => c.getParameterTypes.sameElements(Array(classOf[SparkConf])) @@ -2002,7 +2019,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli lazy val zeroArgumentConstructor = constructors.find { c => c.getParameterTypes.isEmpty } - val listener: SparkListener = { + val listener: SparkListenerInterface = { if (constructorTakingSparkConf.isDefined) { constructorTakingSparkConf.get.newInstance(conf) } else if (zeroArgumentConstructor.isDefined) { @@ -2380,9 +2397,8 @@ object SparkContext extends Logging { } catch { // TODO: Enumerate the exact reasons why it can fail // But irrespective of it, it means we cannot proceed ! - case e: Exception => { + case e: Exception => throw new SparkException("YARN mode not available ?", e) - } } val backend = try { val clazz = @@ -2390,9 +2406,8 @@ object SparkContext extends Logging { val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { - case e: Exception => { + case e: Exception => throw new SparkException("YARN mode not available ?", e) - } } scheduler.initialize(backend) (backend, scheduler) @@ -2404,9 +2419,8 @@ object SparkContext extends Logging { cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] } catch { - case e: Exception => { + case e: Exception => throw new SparkException("YARN mode not available ?", e) - } } val backend = try { @@ -2415,9 +2429,8 @@ object SparkContext extends Logging { val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { - case e: Exception => { + case e: Exception => throw new SparkException("YARN mode not available ?", e) - } } scheduler.initialize(backend) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 700e2cb3f9..3d11db7461 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -101,14 +101,13 @@ class SparkEnv ( // We only need to delete the tmp dir create by driver, because sparkFilesDir is point to the // current working dir in executor which we do not need to delete. driverTmpDirToDelete match { - case Some(path) => { + case Some(path) => try { Utils.deleteRecursively(new File(path)) } catch { case e: Exception => logWarning(s"Exception while deleting Spark temp dir: $path", e) } - } case None => // We just need to delete tmp dir created by driver, so do nothing on executor } } @@ -314,7 +313,8 @@ object SparkEnv extends Logging { UnifiedMemoryManager(conf, numUsableCores) } - val blockTransferService = new NettyBlockTransferService(conf, securityManager, numUsableCores) + val blockTransferService = + new NettyBlockTransferService(conf, securityManager, hostname, numUsableCores) val blockManagerMaster = new BlockManagerMaster(registerOrLookupEndpoint( BlockManagerMaster.DRIVER_ENDPOINT_NAME, diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala index 34ee3a48f8..52c4656c27 100644 --- a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala +++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala @@ -17,6 +17,8 @@ package org.apache.spark +import org.apache.spark.scheduler.TaskSchedulerImpl + /** * Low-level status reporting APIs for monitoring job and stage progress. * @@ -104,4 +106,22 @@ class SparkStatusTracker private[spark] (sc: SparkContext) { } } } + + /** + * Returns information of all known executors, including host, port, cacheSize, numRunningTasks. + */ + def getExecutorInfos: Array[SparkExecutorInfo] = { + val executorIdToRunningTasks: Map[String, Int] = + sc.taskScheduler.asInstanceOf[TaskSchedulerImpl].runningTasksByExecutors() + + sc.getExecutorStorageStatus.map { status => + val bmId = status.blockManagerId + new SparkExecutorInfoImpl( + bmId.host, + bmId.port, + status.cacheSize, + executorIdToRunningTasks.getOrElse(bmId.executorId, 0) + ) + } + } } diff --git a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala index e5c7c8d0db..c1f24a6377 100644 --- a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala +++ b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala @@ -18,18 +18,25 @@ package org.apache.spark private class SparkJobInfoImpl ( - val jobId: Int, - val stageIds: Array[Int], - val status: JobExecutionStatus) - extends SparkJobInfo + val jobId: Int, + val stageIds: Array[Int], + val status: JobExecutionStatus) + extends SparkJobInfo private class SparkStageInfoImpl( - val stageId: Int, - val currentAttemptId: Int, - val submissionTime: Long, - val name: String, - val numTasks: Int, - val numActiveTasks: Int, - val numCompletedTasks: Int, - val numFailedTasks: Int) - extends SparkStageInfo + val stageId: Int, + val currentAttemptId: Int, + val submissionTime: Long, + val name: String, + val numTasks: Int, + val numActiveTasks: Int, + val numCompletedTasks: Int, + val numFailedTasks: Int) + extends SparkStageInfo + +private class SparkExecutorInfoImpl( + val host: String, + val port: Int, + val cacheSize: Long, + val numRunningTasks: Int) + extends SparkExecutorInfo diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 0c1a1dec30..5b2fca4b2d 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -18,6 +18,7 @@ package org.apache.spark import java.io.Serializable +import java.util.Properties import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics @@ -64,7 +65,7 @@ object TaskContext { * An empty task context that does not represent an actual task. */ private[spark] def empty(): TaskContextImpl = { - new TaskContextImpl(0, 0, 0, 0, null, null) + new TaskContextImpl(0, 0, 0, 0, null, new Properties, null) } } @@ -170,6 +171,12 @@ abstract class TaskContext extends Serializable { */ def taskAttemptId(): Long + /** + * Get a local property set upstream in the driver, or null if it is missing. See also + * [[org.apache.spark.SparkContext.setLocalProperty]]. + */ + def getLocalProperty(key: String): String + @DeveloperApi def taskMetrics(): TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 87dc7f30e7..8b407f9771 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -17,6 +17,8 @@ package org.apache.spark +import java.util.Properties + import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics @@ -32,6 +34,7 @@ private[spark] class TaskContextImpl( override val taskAttemptId: Long, override val attemptNumber: Int, override val taskMemoryManager: TaskMemoryManager, + localProperties: Properties, @transient private val metricsSystem: MetricsSystem, initialAccumulators: Seq[Accumulator[_]] = InternalAccumulator.createAll()) extends TaskContext @@ -119,6 +122,8 @@ private[spark] class TaskContextImpl( override def isInterrupted(): Boolean = interrupted + override def getLocalProperty(key: String): String = localProperties.getProperty(key) + override def getMetricsSources(sourceName: String): Seq[Source] = metricsSystem.getSourcesByName(sourceName) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index e080f91f50..2897272a8b 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -461,10 +461,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) fromRDD(rdd.partitionBy(partitioner)) /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. - */ + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. + */ def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = fromRDD(rdd.join(other, partitioner)) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index d362c40b7a..dfd91ae338 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -295,13 +295,14 @@ class JavaSparkContext(val sc: SparkContext) new JavaRDD(sc.binaryRecords(path, recordLength)) } - /** Get an RDD for a Hadoop SequenceFile with given key and value types. - * - * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. - * */ + /** + * Get an RDD for a Hadoop SequenceFile with given key and value types. + * + * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each + * record, directly caching the returned RDD will create many references to the same object. + * If you plan to directly cache Hadoop writable objects, you should first copy them using + * a `map` function. + */ def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V], @@ -312,13 +313,14 @@ class JavaSparkContext(val sc: SparkContext) new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minPartitions)) } - /** Get an RDD for a Hadoop SequenceFile. - * - * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. - */ + /** + * Get an RDD for a Hadoop SequenceFile. + * + * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each + * record, directly caching the returned RDD will create many references to the same object. + * If you plan to directly cache Hadoop writable objects, you should first copy them using + * a `map` function. + */ def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) @@ -411,13 +413,14 @@ class JavaSparkContext(val sc: SparkContext) new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } - /** Get an RDD for a Hadoop file with an arbitrary InputFormat. - * - * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. - */ + /** + * Get an RDD for a Hadoop file with an arbitrary InputFormat. + * + * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each + * record, directly caching the returned RDD will create many references to the same object. + * If you plan to directly cache Hadoop writable objects, you should first copy them using + * a `map` function. + */ def hadoopFile[K, V, F <: InputFormat[K, V]]( path: String, inputFormatClass: Class[F], @@ -431,13 +434,14 @@ class JavaSparkContext(val sc: SparkContext) new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } - /** Get an RDD for a Hadoop file with an arbitrary InputFormat - * - * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. - */ + /** + * Get an RDD for a Hadoop file with an arbitrary InputFormat + * + * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each + * record, directly caching the returned RDD will create many references to the same object. + * If you plan to directly cache Hadoop writable objects, you should first copy them using + * a `map` function. + */ def hadoopFile[K, V, F <: InputFormat[K, V]]( path: String, inputFormatClass: Class[F], diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index 6f6730690f..6259bead3e 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -134,11 +134,10 @@ private[python] class JavaToWritableConverter extends Converter[Any, Writable] { mapWritable.put(convertToWritable(k), convertToWritable(v)) } mapWritable - case array: Array[Any] => { + case array: Array[Any] => val arrayWriteable = new ArrayWritable(classOf[Writable]) arrayWriteable.set(array.map(convertToWritable(_))) arrayWriteable - } case other => throw new SparkException( s"Data of type ${other.getClass.getName} cannot be used") } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index f423b2ee56..ab5b6c8380 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -59,7 +59,7 @@ private[spark] class PythonRDD( val asJavaRDD: JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { - val runner = new PythonRunner(func, bufferSize, reuse_worker) + val runner = PythonRunner(func, bufferSize, reuse_worker) runner.compute(firstParent.iterator(split, context), split.index, context) } } @@ -78,17 +78,41 @@ private[spark] case class PythonFunction( accumulator: Accumulator[JList[Array[Byte]]]) /** - * A helper class to run Python UDFs in Spark. + * A wrapper for chained Python functions (from bottom to top). + * @param funcs + */ +private[spark] case class ChainedPythonFunctions(funcs: Seq[PythonFunction]) + +private[spark] object PythonRunner { + def apply(func: PythonFunction, bufferSize: Int, reuse_worker: Boolean): PythonRunner = { + new PythonRunner( + Seq(ChainedPythonFunctions(Seq(func))), bufferSize, reuse_worker, false, Array(Array(0))) + } +} + +/** + * A helper class to run Python mapPartition/UDFs in Spark. + * + * funcs is a list of independent Python functions, each one of them is a list of chained Python + * functions (from bottom to top). */ private[spark] class PythonRunner( - func: PythonFunction, + funcs: Seq[ChainedPythonFunctions], bufferSize: Int, - reuse_worker: Boolean) + reuse_worker: Boolean, + isUDF: Boolean, + argOffsets: Array[Array[Int]]) extends Logging { - private val envVars = func.envVars - private val pythonExec = func.pythonExec - private val accumulator = func.accumulator + require(funcs.length == argOffsets.length, "argOffsets should have the same length as funcs") + + // All the Python functions should have the same exec, version and envvars. + private val envVars = funcs.head.funcs.head.envVars + private val pythonExec = funcs.head.funcs.head.pythonExec + private val pythonVer = funcs.head.funcs.head.pythonVer + + // TODO: support accumulator in multiple UDF + private val accumulator = funcs.head.funcs.head.accumulator def compute( inputIterator: Iterator[_], @@ -228,10 +252,8 @@ private[spark] class PythonRunner( @volatile private var _exception: Exception = null - private val pythonVer = func.pythonVer - private val pythonIncludes = func.pythonIncludes - private val broadcastVars = func.broadcastVars - private val command = func.command + private val pythonIncludes = funcs.flatMap(_.funcs.flatMap(_.pythonIncludes.asScala)).toSet + private val broadcastVars = funcs.flatMap(_.funcs.flatMap(_.broadcastVars.asScala)) setDaemon(true) @@ -256,13 +278,13 @@ private[spark] class PythonRunner( // sparkFilesDir PythonRDD.writeUTF(SparkFiles.getRootDirectory(), dataOut) // Python includes (*.zip and *.egg files) - dataOut.writeInt(pythonIncludes.size()) - for (include <- pythonIncludes.asScala) { + dataOut.writeInt(pythonIncludes.size) + for (include <- pythonIncludes) { PythonRDD.writeUTF(include, dataOut) } // Broadcast variables val oldBids = PythonRDD.getWorkerBroadcasts(worker) - val newBids = broadcastVars.asScala.map(_.id).toSet + val newBids = broadcastVars.map(_.id).toSet // number of different broadcasts val toRemove = oldBids.diff(newBids) val cnt = toRemove.size + newBids.diff(oldBids).size @@ -272,7 +294,7 @@ private[spark] class PythonRunner( dataOut.writeLong(- bid - 1) // bid >= 0 oldBids.remove(bid) } - for (broadcast <- broadcastVars.asScala) { + for (broadcast <- broadcastVars) { if (!oldBids.contains(broadcast.id)) { // send new broadcast dataOut.writeLong(broadcast.id) @@ -282,8 +304,26 @@ private[spark] class PythonRunner( } dataOut.flush() // Serialized command: - dataOut.writeInt(command.length) - dataOut.write(command) + if (isUDF) { + dataOut.writeInt(1) + dataOut.writeInt(funcs.length) + funcs.zip(argOffsets).foreach { case (chained, offsets) => + dataOut.writeInt(offsets.length) + offsets.foreach { offset => + dataOut.writeInt(offset) + } + dataOut.writeInt(chained.funcs.length) + chained.funcs.foreach { f => + dataOut.writeInt(f.command.length) + dataOut.write(f.command) + } + } + } else { + dataOut.writeInt(0) + val command = funcs.head.funcs.head.command + dataOut.writeInt(command.length) + dataOut.write(command) + } // Data values PythonRDD.writeIteratorToStream(inputIterator, dataOut) dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION) @@ -413,6 +453,10 @@ private[spark] object PythonRDD extends Logging { serveIterator(rdd.collect().iterator, s"serve RDD ${rdd.id}") } + def toLocalIteratorAndServe[T](rdd: RDD[T]): Int = { + serveIterator(rdd.toLocalIterator, s"serve toLocalIterator") + } + def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { val file = new DataInputStream(new FileInputStream(filename)) @@ -426,7 +470,7 @@ private[spark] object PythonRDD extends Logging { objs.append(obj) } } catch { - case eof: EOFException => {} + case eof: EOFException => // No-op } JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } finally { diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 588a57e65f..606ba6ef86 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -17,21 +17,16 @@ package org.apache.spark.api.r -import java.io._ -import java.net.{InetAddress, ServerSocket} -import java.util.{Arrays, Map => JMap} +import java.util.{Map => JMap} import scala.collection.JavaConverters._ -import scala.io.Source import scala.reflect.ClassTag -import scala.util.Try import org.apache.spark._ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.util.Utils private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( parent: RDD[T], @@ -42,188 +37,16 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( packageNames: Array[Byte], broadcastVars: Array[Broadcast[Object]]) extends RDD[U](parent) with Logging { - protected var dataStream: DataInputStream = _ - private var bootTime: Double = _ override def getPartitions: Array[Partition] = parent.partitions override def compute(partition: Partition, context: TaskContext): Iterator[U] = { - - // Timing start - bootTime = System.currentTimeMillis / 1000.0 + val runner = new RRunner[U]( + func, deserializer, serializer, packageNames, broadcastVars, numPartitions) // The parent may be also an RRDD, so we should launch it first. val parentIterator = firstParent[T].iterator(partition, context) - // we expect two connections - val serverSocket = new ServerSocket(0, 2, InetAddress.getByName("localhost")) - val listenPort = serverSocket.getLocalPort() - - // The stdout/stderr is shared by multiple tasks, because we use one daemon - // to launch child process as worker. - val errThread = RRDD.createRWorker(listenPort) - - // We use two sockets to separate input and output, then it's easy to manage - // the lifecycle of them to avoid deadlock. - // TODO: optimize it to use one socket - - // the socket used to send out the input of task - serverSocket.setSoTimeout(10000) - val inSocket = serverSocket.accept() - startStdinThread(inSocket.getOutputStream(), parentIterator, partition.index) - - // the socket used to receive the output of task - val outSocket = serverSocket.accept() - val inputStream = new BufferedInputStream(outSocket.getInputStream) - dataStream = new DataInputStream(inputStream) - serverSocket.close() - - try { - - return new Iterator[U] { - def next(): U = { - val obj = _nextObj - if (hasNext) { - _nextObj = read() - } - obj - } - - var _nextObj = read() - - def hasNext(): Boolean = { - val hasMore = (_nextObj != null) - if (!hasMore) { - dataStream.close() - } - hasMore - } - } - } catch { - case e: Exception => - throw new SparkException("R computation failed with\n " + errThread.getLines()) - } - } - - /** - * Start a thread to write RDD data to the R process. - */ - private def startStdinThread[T]( - output: OutputStream, - iter: Iterator[T], - partition: Int): Unit = { - - val env = SparkEnv.get - val taskContext = TaskContext.get() - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - val stream = new BufferedOutputStream(output, bufferSize) - - new Thread("writer for R") { - override def run(): Unit = { - try { - SparkEnv.set(env) - TaskContext.setTaskContext(taskContext) - val dataOut = new DataOutputStream(stream) - dataOut.writeInt(partition) - - SerDe.writeString(dataOut, deserializer) - SerDe.writeString(dataOut, serializer) - - dataOut.writeInt(packageNames.length) - dataOut.write(packageNames) - - dataOut.writeInt(func.length) - dataOut.write(func) - - dataOut.writeInt(broadcastVars.length) - broadcastVars.foreach { broadcast => - // TODO(shivaram): Read a Long in R to avoid this cast - dataOut.writeInt(broadcast.id.toInt) - // TODO: Pass a byte array from R to avoid this cast ? - val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]] - dataOut.writeInt(broadcastByteArr.length) - dataOut.write(broadcastByteArr) - } - - dataOut.writeInt(numPartitions) - - if (!iter.hasNext) { - dataOut.writeInt(0) - } else { - dataOut.writeInt(1) - } - - val printOut = new PrintStream(stream) - - def writeElem(elem: Any): Unit = { - if (deserializer == SerializationFormats.BYTE) { - val elemArr = elem.asInstanceOf[Array[Byte]] - dataOut.writeInt(elemArr.length) - dataOut.write(elemArr) - } else if (deserializer == SerializationFormats.ROW) { - dataOut.write(elem.asInstanceOf[Array[Byte]]) - } else if (deserializer == SerializationFormats.STRING) { - // write string(for StringRRDD) - // scalastyle:off println - printOut.println(elem) - // scalastyle:on println - } - } - - for (elem <- iter) { - elem match { - case (key, value) => - writeElem(key) - writeElem(value) - case _ => - writeElem(elem) - } - } - stream.flush() - } catch { - // TODO: We should propogate this error to the task thread - case e: Exception => - logError("R Writer thread got an exception", e) - } finally { - Try(output.close()) - } - } - }.start() - } - - protected def readData(length: Int): U - - protected def read(): U = { - try { - val length = dataStream.readInt() - - length match { - case SpecialLengths.TIMING_DATA => - // Timing data from R worker - val boot = dataStream.readDouble - bootTime - val init = dataStream.readDouble - val broadcast = dataStream.readDouble - val input = dataStream.readDouble - val compute = dataStream.readDouble - val output = dataStream.readDouble - logInfo( - ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " + - "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " + - "total = %.3f s").format( - boot, - init, - broadcast, - input, - compute, - output, - boot + init + broadcast + input + compute + output)) - read() - case length if length >= 0 => - readData(length) - } - } catch { - case eof: EOFException => - throw new SparkException("R worker exited unexpectedly (cranshed)", eof) - } + runner.compute(parentIterator, partition.index, context) } } @@ -242,19 +65,6 @@ private class PairwiseRRDD[T: ClassTag]( parent, numPartitions, hashFunc, deserializer, SerializationFormats.BYTE, packageNames, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - - override protected def readData(length: Int): (Int, Array[Byte]) = { - length match { - case length if length == 2 => - val hashedKey = dataStream.readInt() - val contentPairsLength = dataStream.readInt() - val contentPairs = new Array[Byte](contentPairsLength) - dataStream.readFully(contentPairs) - (hashedKey, contentPairs) - case _ => null - } - } - lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = JavaPairRDD.fromRDD(this) } @@ -271,17 +81,6 @@ private class RRDD[T: ClassTag]( extends BaseRRDD[T, Array[Byte]]( parent, -1, func, deserializer, serializer, packageNames, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - - override protected def readData(length: Int): Array[Byte] = { - length match { - case length if length > 0 => - val obj = new Array[Byte](length) - dataStream.readFully(obj) - obj - case _ => null - } - } - lazy val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) } @@ -297,55 +96,10 @@ private class StringRRDD[T: ClassTag]( extends BaseRRDD[T, String]( parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { - - override protected def readData(length: Int): String = { - length match { - case length if length > 0 => - SerDe.readStringBytes(dataStream, length) - case _ => null - } - } - lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) } -private object SpecialLengths { - val TIMING_DATA = -1 -} - -private[r] class BufferedStreamThread( - in: InputStream, - name: String, - errBufferSize: Int) extends Thread(name) with Logging { - val lines = new Array[String](errBufferSize) - var lineIdx = 0 - override def run() { - for (line <- Source.fromInputStream(in).getLines) { - synchronized { - lines(lineIdx) = line - lineIdx = (lineIdx + 1) % errBufferSize - } - logInfo(line) - } - } - - def getLines(): String = synchronized { - (0 until errBufferSize).filter { x => - lines((x + lineIdx) % errBufferSize) != null - }.map { x => - lines((x + lineIdx) % errBufferSize) - }.mkString("\n") - } -} - private[r] object RRDD { - // Because forking processes from Java is expensive, we prefer to launch - // a single R daemon (daemon.R) and tell it to fork new workers for our tasks. - // This daemon currently only works on UNIX-based systems now, so we should - // also fall back to launching workers (worker.R) directly. - private[this] var errThread: BufferedStreamThread = _ - private[this] var daemonChannel: DataOutputStream = _ - def createSparkContext( master: String, appName: String, @@ -353,7 +107,6 @@ private[r] object RRDD { jars: Array[String], sparkEnvirMap: JMap[Object, Object], sparkExecutorEnvMap: JMap[Object, Object]): JavaSparkContext = { - val sparkConf = new SparkConf().setAppName(appName) .setSparkHome(sparkHome) @@ -381,83 +134,10 @@ private[r] object RRDD { } /** - * Start a thread to print the process's stderr to ours - */ - private def startStdoutThread(proc: Process): BufferedStreamThread = { - val BUFFER_SIZE = 100 - val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE) - thread.setDaemon(true) - thread.start() - thread - } - - private def createRProcess(port: Int, script: String): BufferedStreamThread = { - // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command", - // but kept here for backward compatibility. - val sparkConf = SparkEnv.get.conf - var rCommand = sparkConf.get("spark.sparkr.r.command", "Rscript") - rCommand = sparkConf.get("spark.r.command", rCommand) - - val rOptions = "--vanilla" - val rLibDir = RUtils.sparkRPackagePath(isDriver = false) - val rExecScript = rLibDir(0) + "/SparkR/worker/" + script - val pb = new ProcessBuilder(Arrays.asList(rCommand, rOptions, rExecScript)) - // Unset the R_TESTS environment variable for workers. - // This is set by R CMD check as startup.Rs - // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R) - // and confuses worker script which tries to load a non-existent file - pb.environment().put("R_TESTS", "") - pb.environment().put("SPARKR_RLIBDIR", rLibDir.mkString(",")) - pb.environment().put("SPARKR_WORKER_PORT", port.toString) - pb.redirectErrorStream(true) // redirect stderr into stdout - val proc = pb.start() - val errThread = startStdoutThread(proc) - errThread - } - - /** - * ProcessBuilder used to launch worker R processes. - */ - def createRWorker(port: Int): BufferedStreamThread = { - val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) - if (!Utils.isWindows && useDaemon) { - synchronized { - if (daemonChannel == null) { - // we expect one connections - val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) - val daemonPort = serverSocket.getLocalPort - errThread = createRProcess(daemonPort, "daemon.R") - // the socket used to send out the input of task - serverSocket.setSoTimeout(10000) - val sock = serverSocket.accept() - daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) - serverSocket.close() - } - try { - daemonChannel.writeInt(port) - daemonChannel.flush() - } catch { - case e: IOException => - // daemon process died - daemonChannel.close() - daemonChannel = null - errThread = null - // fail the current task, retry by scheduler - throw e - } - errThread - } - } else { - createRProcess(port, "worker.R") - } - } - - /** * Create an RRDD given a sequence of byte arrays. Used to create RRDD when `parallelize` is * called from R. */ def createRDDFromArray(jsc: JavaSparkContext, arr: Array[Array[Byte]]): JavaRDD[Array[Byte]] = { JavaRDD.fromRDD(jsc.sc.parallelize(arr, arr.length)) } - } diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala new file mode 100644 index 0000000000..07d1fa2c4a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -0,0 +1,368 @@ +/* + * 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.api.r + +import java.io._ +import java.net.{InetAddress, ServerSocket} +import java.util.Arrays + +import scala.io.Source +import scala.util.Try + +import org.apache.spark._ +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * A helper class to run R UDFs in Spark. + */ +private[spark] class RRunner[U]( + func: Array[Byte], + deserializer: String, + serializer: String, + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + numPartitions: Int = -1) + extends Logging { + private var bootTime: Double = _ + private var dataStream: DataInputStream = _ + val readData = numPartitions match { + case -1 => + serializer match { + case SerializationFormats.STRING => readStringData _ + case _ => readByteArrayData _ + } + case _ => readShuffledData _ + } + + def compute( + inputIterator: Iterator[_], + partitionIndex: Int, + context: TaskContext): Iterator[U] = { + // Timing start + bootTime = System.currentTimeMillis / 1000.0 + + // we expect two connections + val serverSocket = new ServerSocket(0, 2, InetAddress.getByName("localhost")) + val listenPort = serverSocket.getLocalPort() + + // The stdout/stderr is shared by multiple tasks, because we use one daemon + // to launch child process as worker. + val errThread = RRunner.createRWorker(listenPort) + + // We use two sockets to separate input and output, then it's easy to manage + // the lifecycle of them to avoid deadlock. + // TODO: optimize it to use one socket + + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + val inSocket = serverSocket.accept() + startStdinThread(inSocket.getOutputStream(), inputIterator, partitionIndex) + + // the socket used to receive the output of task + val outSocket = serverSocket.accept() + val inputStream = new BufferedInputStream(outSocket.getInputStream) + dataStream = new DataInputStream(inputStream) + serverSocket.close() + + try { + return new Iterator[U] { + def next(): U = { + val obj = _nextObj + if (hasNext) { + _nextObj = read() + } + obj + } + + var _nextObj = read() + + def hasNext(): Boolean = { + val hasMore = (_nextObj != null) + if (!hasMore) { + dataStream.close() + } + hasMore + } + } + } catch { + case e: Exception => + throw new SparkException("R computation failed with\n " + errThread.getLines()) + } + } + + /** + * Start a thread to write RDD data to the R process. + */ + private def startStdinThread( + output: OutputStream, + iter: Iterator[_], + partitionIndex: Int): Unit = { + val env = SparkEnv.get + val taskContext = TaskContext.get() + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val stream = new BufferedOutputStream(output, bufferSize) + + new Thread("writer for R") { + override def run(): Unit = { + try { + SparkEnv.set(env) + TaskContext.setTaskContext(taskContext) + val dataOut = new DataOutputStream(stream) + dataOut.writeInt(partitionIndex) + + SerDe.writeString(dataOut, deserializer) + SerDe.writeString(dataOut, serializer) + + dataOut.writeInt(packageNames.length) + dataOut.write(packageNames) + + dataOut.writeInt(func.length) + dataOut.write(func) + + dataOut.writeInt(broadcastVars.length) + broadcastVars.foreach { broadcast => + // TODO(shivaram): Read a Long in R to avoid this cast + dataOut.writeInt(broadcast.id.toInt) + // TODO: Pass a byte array from R to avoid this cast ? + val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]] + dataOut.writeInt(broadcastByteArr.length) + dataOut.write(broadcastByteArr) + } + + dataOut.writeInt(numPartitions) + + if (!iter.hasNext) { + dataOut.writeInt(0) + } else { + dataOut.writeInt(1) + } + + val printOut = new PrintStream(stream) + + def writeElem(elem: Any): Unit = { + if (deserializer == SerializationFormats.BYTE) { + val elemArr = elem.asInstanceOf[Array[Byte]] + dataOut.writeInt(elemArr.length) + dataOut.write(elemArr) + } else if (deserializer == SerializationFormats.ROW) { + dataOut.write(elem.asInstanceOf[Array[Byte]]) + } else if (deserializer == SerializationFormats.STRING) { + // write string(for StringRRDD) + // scalastyle:off println + printOut.println(elem) + // scalastyle:on println + } + } + + for (elem <- iter) { + elem match { + case (key, value) => + writeElem(key) + writeElem(value) + case _ => + writeElem(elem) + } + } + stream.flush() + } catch { + // TODO: We should propagate this error to the task thread + case e: Exception => + logError("R Writer thread got an exception", e) + } finally { + Try(output.close()) + } + } + }.start() + } + + private def read(): U = { + try { + val length = dataStream.readInt() + + length match { + case SpecialLengths.TIMING_DATA => + // Timing data from R worker + val boot = dataStream.readDouble - bootTime + val init = dataStream.readDouble + val broadcast = dataStream.readDouble + val input = dataStream.readDouble + val compute = dataStream.readDouble + val output = dataStream.readDouble + logInfo( + ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " + + "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " + + "total = %.3f s").format( + boot, + init, + broadcast, + input, + compute, + output, + boot + init + broadcast + input + compute + output)) + read() + case length if length >= 0 => + readData(length).asInstanceOf[U] + } + } catch { + case eof: EOFException => + throw new SparkException("R worker exited unexpectedly (cranshed)", eof) + } + } + + private def readShuffledData(length: Int): (Int, Array[Byte]) = { + length match { + case length if length == 2 => + val hashedKey = dataStream.readInt() + val contentPairsLength = dataStream.readInt() + val contentPairs = new Array[Byte](contentPairsLength) + dataStream.readFully(contentPairs) + (hashedKey, contentPairs) + case _ => null + } + } + + private def readByteArrayData(length: Int): Array[Byte] = { + length match { + case length if length > 0 => + val obj = new Array[Byte](length) + dataStream.readFully(obj) + obj + case _ => null + } + } + + private def readStringData(length: Int): String = { + length match { + case length if length > 0 => + SerDe.readStringBytes(dataStream, length) + case _ => null + } + } +} + +private object SpecialLengths { + val TIMING_DATA = -1 +} + +private[r] class BufferedStreamThread( + in: InputStream, + name: String, + errBufferSize: Int) extends Thread(name) with Logging { + val lines = new Array[String](errBufferSize) + var lineIdx = 0 + override def run() { + for (line <- Source.fromInputStream(in).getLines) { + synchronized { + lines(lineIdx) = line + lineIdx = (lineIdx + 1) % errBufferSize + } + logInfo(line) + } + } + + def getLines(): String = synchronized { + (0 until errBufferSize).filter { x => + lines((x + lineIdx) % errBufferSize) != null + }.map { x => + lines((x + lineIdx) % errBufferSize) + }.mkString("\n") + } +} + +private[r] object RRunner { + // Because forking processes from Java is expensive, we prefer to launch + // a single R daemon (daemon.R) and tell it to fork new workers for our tasks. + // This daemon currently only works on UNIX-based systems now, so we should + // also fall back to launching workers (worker.R) directly. + private[this] var errThread: BufferedStreamThread = _ + private[this] var daemonChannel: DataOutputStream = _ + + /** + * Start a thread to print the process's stderr to ours + */ + private def startStdoutThread(proc: Process): BufferedStreamThread = { + val BUFFER_SIZE = 100 + val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE) + thread.setDaemon(true) + thread.start() + thread + } + + private def createRProcess(port: Int, script: String): BufferedStreamThread = { + // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command", + // but kept here for backward compatibility. + val sparkConf = SparkEnv.get.conf + var rCommand = sparkConf.get("spark.sparkr.r.command", "Rscript") + rCommand = sparkConf.get("spark.r.command", rCommand) + + val rOptions = "--vanilla" + val rLibDir = RUtils.sparkRPackagePath(isDriver = false) + val rExecScript = rLibDir(0) + "/SparkR/worker/" + script + val pb = new ProcessBuilder(Arrays.asList(rCommand, rOptions, rExecScript)) + // Unset the R_TESTS environment variable for workers. + // This is set by R CMD check as startup.Rs + // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R) + // and confuses worker script which tries to load a non-existent file + pb.environment().put("R_TESTS", "") + pb.environment().put("SPARKR_RLIBDIR", rLibDir.mkString(",")) + pb.environment().put("SPARKR_WORKER_PORT", port.toString) + pb.redirectErrorStream(true) // redirect stderr into stdout + val proc = pb.start() + val errThread = startStdoutThread(proc) + errThread + } + + /** + * ProcessBuilder used to launch worker R processes. + */ + def createRWorker(port: Int): BufferedStreamThread = { + val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) + if (!Utils.isWindows && useDaemon) { + synchronized { + if (daemonChannel == null) { + // we expect one connections + val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) + val daemonPort = serverSocket.getLocalPort + errThread = createRProcess(daemonPort, "daemon.R") + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + val sock = serverSocket.accept() + daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) + serverSocket.close() + } + try { + daemonChannel.writeInt(port) + daemonChannel.flush() + } catch { + case e: IOException => + // daemon process died + daemonChannel.close() + daemonChannel = null + errThread = null + // fail the current task, retry by scheduler + throw e + } + errThread + } + } else { + createRProcess(port, "worker.R") + } + } +} diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index e5e6a9e4a8..632b0ae9c2 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -30,7 +30,7 @@ import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BroadcastBlockId, StorageLevel} import org.apache.spark.util.{ByteBufferInputStream, Utils} -import org.apache.spark.util.io.{ByteArrayChunkOutputStream, ChunkedByteBuffer} +import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} /** * A BitTorrent-like implementation of [[org.apache.spark.broadcast.Broadcast]]. @@ -228,12 +228,12 @@ private object TorrentBroadcast extends Logging { blockSize: Int, serializer: Serializer, compressionCodec: Option[CompressionCodec]): Array[ByteBuffer] = { - val bos = new ByteArrayChunkOutputStream(blockSize) - val out: OutputStream = compressionCodec.map(c => c.compressedOutputStream(bos)).getOrElse(bos) + val cbbos = new ChunkedByteBufferOutputStream(blockSize, ByteBuffer.allocate) + val out = compressionCodec.map(c => c.compressedOutputStream(cbbos)).getOrElse(cbbos) val ser = serializer.newInstance() val serOut = ser.serializeStream(out) serOut.writeObject[T](obj).close() - bos.toArrays.map(ByteBuffer.wrap) + cbbos.toChunkedByteBuffer.getChunks() } def unBlockifyObject[T: ClassTag]( diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 06b7b388ca..cda9d38c6a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -74,13 +74,12 @@ class SparkHadoopUtil extends Logging { } } + /** - * Return an appropriate (subclass) of Configuration. Creating config can initializes some Hadoop - * subsystems. + * Appends S3-specific, spark.hadoop.*, and spark.buffer.size configurations to a Hadoop + * configuration. */ - def newConfiguration(conf: SparkConf): Configuration = { - val hadoopConf = new Configuration() - + def appendS3AndSparkHadoopConfigurations(conf: SparkConf, hadoopConf: Configuration): Unit = { // Note: this null check is around more than just access to the "conf" object to maintain // the behavior of the old implementation of this code, for backwards compatibility. if (conf != null) { @@ -106,7 +105,15 @@ class SparkHadoopUtil extends Logging { val bufferSize = conf.get("spark.buffer.size", "65536") hadoopConf.set("io.file.buffer.size", bufferSize) } + } + /** + * Return an appropriate (subclass) of Configuration. Creating config can initializes some Hadoop + * subsystems. + */ + def newConfiguration(conf: SparkConf): Configuration = { + val hadoopConf = new Configuration() + appendS3AndSparkHadoopConfigurations(conf, hadoopConf) hadoopConf } @@ -145,10 +152,9 @@ class SparkHadoopUtil extends Logging { val baselineBytesRead = f() Some(() => f() - baselineBytesRead) } catch { - case e @ (_: NoSuchMethodException | _: ClassNotFoundException) => { + case e @ (_: NoSuchMethodException | _: ClassNotFoundException) => logDebug("Couldn't find method for retrieving thread-level FileSystem input data", e) None - } } } @@ -167,10 +173,9 @@ class SparkHadoopUtil extends Logging { val baselineBytesWritten = f() Some(() => f() - baselineBytesWritten) } catch { - case e @ (_: NoSuchMethodException | _: ClassNotFoundException) => { + case e @ (_: NoSuchMethodException | _: ClassNotFoundException) => logDebug("Couldn't find method for retrieving thread-level FileSystem output data", e) None - } } } @@ -308,7 +313,7 @@ class SparkHadoopUtil extends Logging { */ def substituteHadoopVariables(text: String, hadoopConf: Configuration): String = { text match { - case HADOOP_CONF_PATTERN(matched) => { + case HADOOP_CONF_PATTERN(matched) => logDebug(text + " matched " + HADOOP_CONF_PATTERN) val key = matched.substring(13, matched.length() - 1) // remove ${hadoopconf- .. } val eval = Option[String](hadoopConf.get(key)) @@ -323,11 +328,9 @@ class SparkHadoopUtil extends Logging { // Continue to substitute more variables. substituteHadoopVariables(eval.get, hadoopConf) } - } - case _ => { + case _ => logDebug(text + " didn't match " + HADOOP_CONF_PATTERN) text - } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 4049fc0c41..926e1ff7a8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -441,7 +441,6 @@ object SparkSubmit { OptionAssigner(args.deployMode, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.submit.deployMode"), OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"), - OptionAssigner(args.jars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars"), OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars.ivy"), OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.driver.memory"), @@ -452,27 +451,15 @@ object SparkSubmit { OptionAssigner(args.driverExtraLibraryPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.driver.extraLibraryPath"), - // Yarn client only - OptionAssigner(args.queue, YARN, CLIENT, sysProp = "spark.yarn.queue"), + // Yarn only + OptionAssigner(args.queue, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.queue"), OptionAssigner(args.numExecutors, YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.instances"), - OptionAssigner(args.files, YARN, CLIENT, sysProp = "spark.yarn.dist.files"), - OptionAssigner(args.archives, YARN, CLIENT, sysProp = "spark.yarn.dist.archives"), - OptionAssigner(args.principal, YARN, CLIENT, sysProp = "spark.yarn.principal"), - OptionAssigner(args.keytab, YARN, CLIENT, sysProp = "spark.yarn.keytab"), - - // Yarn cluster only - OptionAssigner(args.name, YARN, CLUSTER, clOption = "--name"), - OptionAssigner(args.driverMemory, YARN, CLUSTER, clOption = "--driver-memory"), - OptionAssigner(args.driverCores, YARN, CLUSTER, clOption = "--driver-cores"), - OptionAssigner(args.queue, YARN, CLUSTER, clOption = "--queue"), - OptionAssigner(args.executorMemory, YARN, CLUSTER, clOption = "--executor-memory"), - OptionAssigner(args.executorCores, YARN, CLUSTER, clOption = "--executor-cores"), - OptionAssigner(args.files, YARN, CLUSTER, clOption = "--files"), - OptionAssigner(args.archives, YARN, CLUSTER, clOption = "--archives"), - OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), - OptionAssigner(args.principal, YARN, CLUSTER, clOption = "--principal"), - OptionAssigner(args.keytab, YARN, CLUSTER, clOption = "--keytab"), + OptionAssigner(args.jars, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.jars"), + OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.files"), + OptionAssigner(args.archives, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.dist.archives"), + OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.principal"), + OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.keytab"), // Other options OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES, @@ -483,10 +470,11 @@ object SparkSubmit { sysProp = "spark.cores.max"), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.files"), - OptionAssigner(args.jars, STANDALONE | MESOS, CLUSTER, sysProp = "spark.jars"), - OptionAssigner(args.driverMemory, STANDALONE | MESOS, CLUSTER, + OptionAssigner(args.jars, LOCAL, CLIENT, sysProp = "spark.jars"), + OptionAssigner(args.jars, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.jars"), + OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN, CLUSTER, sysProp = "spark.driver.memory"), - OptionAssigner(args.driverCores, STANDALONE | MESOS, CLUSTER, + OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN, CLUSTER, sysProp = "spark.driver.cores"), OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, sysProp = "spark.driver.supervise"), @@ -550,6 +538,10 @@ object SparkSubmit { if (args.isPython) { sysProps.put("spark.yarn.isPython", "true") } + + if (args.pyFiles != null) { + sysProps("spark.submit.pyFiles") = args.pyFiles + } } // assure a keytab is available from any place in a JVM @@ -576,9 +568,6 @@ object SparkSubmit { childMainClass = "org.apache.spark.deploy.yarn.Client" if (args.isPython) { childArgs += ("--primary-py-file", args.primaryResource) - if (args.pyFiles != null) { - childArgs += ("--py-files", args.pyFiles) - } childArgs += ("--class", "org.apache.spark.deploy.PythonRunner") } else if (args.isR) { val mainFile = new Path(args.primaryResource).getName @@ -627,7 +616,8 @@ object SparkSubmit { "spark.jars", "spark.files", "spark.yarn.dist.files", - "spark.yarn.dist.archives") + "spark.yarn.dist.archives", + "spark.yarn.dist.jars") pathConfigs.foreach { config => // Replace old URIs with resolved URIs, if they exist sysProps.get(config).foreach { oldValue => diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala index e584952a9a..94506a0cbb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala @@ -33,7 +33,8 @@ private[spark] trait AppClientListener { /** An application death is an unrecoverable failure condition. */ def dead(reason: String): Unit - def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) + def executorAdded( + fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala index 70f21fbe0d..52e2854961 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -32,8 +32,8 @@ trait LeaderElectionAgent { @DeveloperApi trait LeaderElectable { - def electedLeader() - def revokedLeadership() + def electedLeader(): Unit + def revokedLeadership(): Unit } /** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 01901bbf85..b443e8f051 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -217,7 +217,7 @@ private[deploy] class Master( } override def receive: PartialFunction[Any, Unit] = { - case ElectedLeader => { + case ElectedLeader => val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData(rpcEnv) state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) { RecoveryState.ALIVE @@ -233,16 +233,14 @@ private[deploy] class Master( } }, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) } - } case CompleteRecovery => completeRecovery() - case RevokedLeadership => { + case RevokedLeadership => logError("Leadership has been revoked -- master shutting down.") System.exit(0) - } - case RegisterApplication(description, driver) => { + case RegisterApplication(description, driver) => // TODO Prevent repeated registrations from some driver if (state == RecoveryState.STANDBY) { // ignore, don't send response @@ -255,12 +253,11 @@ private[deploy] class Master( driver.send(RegisteredApplication(app.id, self)) schedule() } - } - case ExecutorStateChanged(appId, execId, state, message, exitStatus) => { + case ExecutorStateChanged(appId, execId, state, message, exitStatus) => val execOption = idToApp.get(appId).flatMap(app => app.executors.get(execId)) execOption match { - case Some(exec) => { + case Some(exec) => val appInfo = idToApp(appId) val oldState = exec.state exec.state = state @@ -298,22 +295,19 @@ private[deploy] class Master( } } } - } case None => logWarning(s"Got status update for unknown executor $appId/$execId") } - } - case DriverStateChanged(driverId, state, exception) => { + case DriverStateChanged(driverId, state, exception) => state match { case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED => removeDriver(driverId, state, exception) case _ => throw new Exception(s"Received unexpected state update for driver $driverId: $state") } - } - case Heartbeat(workerId, worker) => { + case Heartbeat(workerId, worker) => idToWorker.get(workerId) match { case Some(workerInfo) => workerInfo.lastHeartbeat = System.currentTimeMillis() @@ -327,9 +321,8 @@ private[deploy] class Master( " This worker was never registered, so ignoring the heartbeat.") } } - } - case MasterChangeAcknowledged(appId) => { + case MasterChangeAcknowledged(appId) => idToApp.get(appId) match { case Some(app) => logInfo("Application has been re-registered: " + appId) @@ -339,9 +332,8 @@ private[deploy] class Master( } if (canCompleteRecovery) { completeRecovery() } - } - case WorkerSchedulerStateResponse(workerId, executors, driverIds) => { + case WorkerSchedulerStateResponse(workerId, executors, driverIds) => idToWorker.get(workerId) match { case Some(worker) => logInfo("Worker has been re-registered: " + workerId) @@ -367,7 +359,6 @@ private[deploy] class Master( } if (canCompleteRecovery) { completeRecovery() } - } case WorkerLatestState(workerId, executors, driverIds) => idToWorker.get(workerId) match { @@ -397,9 +388,8 @@ private[deploy] class Master( logInfo(s"Received unregister request from application $applicationId") idToApp.get(applicationId).foreach(finishApplication) - case CheckForWorkerTimeOut => { + case CheckForWorkerTimeOut => timeOutDeadWorkers() - } case AttachCompletedRebuildUI(appId) => // An asyncRebuildSparkUI has completed, so need to attach to master webUi @@ -408,7 +398,7 @@ private[deploy] class Master( override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RegisterWorker( - id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl) => { + id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl) => logInfo("Registering worker %s:%d with %d cores, %s RAM".format( workerHost, workerPort, cores, Utils.megabytesToString(memory))) if (state == RecoveryState.STANDBY) { @@ -430,9 +420,8 @@ private[deploy] class Master( + workerAddress)) } } - } - case RequestSubmitDriver(description) => { + case RequestSubmitDriver(description) => if (state != RecoveryState.ALIVE) { val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " + "Can only accept driver submissions in ALIVE state." @@ -451,9 +440,8 @@ private[deploy] class Master( context.reply(SubmitDriverResponse(self, true, Some(driver.id), s"Driver successfully submitted as ${driver.id}")) } - } - case RequestKillDriver(driverId) => { + case RequestKillDriver(driverId) => if (state != RecoveryState.ALIVE) { val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " + s"Can only kill drivers in ALIVE state." @@ -484,9 +472,8 @@ private[deploy] class Master( context.reply(KillDriverResponse(self, driverId, success = false, msg)) } } - } - case RequestDriverStatus(driverId) => { + case RequestDriverStatus(driverId) => if (state != RecoveryState.ALIVE) { val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " + "Can only request driver status in ALIVE state." @@ -501,18 +488,15 @@ private[deploy] class Master( context.reply(DriverStatusResponse(found = false, None, None, None, None)) } } - } - case RequestMasterState => { + case RequestMasterState => context.reply(MasterStateResponse( address.host, address.port, restServerBoundPort, workers.toArray, apps.toArray, completedApps.toArray, drivers.toArray, completedDrivers.toArray, state)) - } - case BoundPortsRequest => { + case BoundPortsRequest => context.reply(BoundPortsResponse(address.port, webUi.boundPort, restServerBoundPort)) - } case RequestExecutors(appId, requestedTotal) => context.reply(handleRequestExecutors(appId, requestedTotal)) @@ -859,10 +843,10 @@ private[deploy] class Master( addressToApp -= app.driver.address if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) - completedApps.take(toRemove).foreach( a => { + completedApps.take(toRemove).foreach { a => Option(appIdToUI.remove(a.id)).foreach { ui => webUi.detachSparkUI(ui) } applicationMetricsSystem.removeSource(a.appSource) - }) + } completedApps.trimStart(toRemove) } completedApps += app // Remember it in our history diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 9cd7458ba0..585e0839d0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -78,7 +78,7 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) { case ("--help") :: tail => printUsageAndExit(0) - case Nil => {} + case Nil => // No-op case _ => printUsageAndExit(1) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala index dddf2be57e..b30bc821b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -40,12 +40,12 @@ abstract class PersistenceEngine { * Defines how the object is serialized and persisted. Implementation will * depend on the store used. */ - def persist(name: String, obj: Object) + def persist(name: String, obj: Object): Unit /** * Defines how the object referred by its name is removed from the store. */ - def unpersist(name: String) + def unpersist(name: String): Unit /** * Gives all objects, matching a prefix. This defines how objects are diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 79f77212fe..af850e4871 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -70,11 +70,10 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer try { Some(serializer.newInstance().deserialize[T](ByteBuffer.wrap(fileData))) } catch { - case e: Exception => { + case e: Exception => logWarning("Exception while reading persisted file, deleting", e) zk.delete().forPath(WORKING_DIR + "/" + filename) None - } } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala index b97805a28b..11e13441ee 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala @@ -76,14 +76,13 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: case ("--help") :: tail => printUsageAndExit(0) - case Nil => { + case Nil => if (masterUrl == null) { // scalastyle:off println System.err.println("--master is required") // scalastyle:on println printUsageAndExit(1) } - } case _ => printUsageAndExit(1) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index a4efafcb27..cba4aaffe2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -29,7 +29,7 @@ import org.apache.spark.launcher.WorkerCommandBuilder import org.apache.spark.util.Utils /** - ** Utilities for running commands with the spark classpath. + * Utilities for running commands with the spark classpath. */ private[deploy] object CommandUtils extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 9c6bc5c62f..aad2e91b25 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -218,7 +218,7 @@ private[deploy] class DriverRunner( } private[deploy] trait Sleeper { - def sleep(seconds: Int) + def sleep(seconds: Int): Unit } // Needed because ProcessBuilder is a final class and cannot be mocked diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index f9c92c3bb9..06066248ea 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -179,16 +179,14 @@ private[deploy] class ExecutorRunner( val message = "Command exited with code " + exitCode worker.send(ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode))) } catch { - case interrupted: InterruptedException => { + case interrupted: InterruptedException => logInfo("Runner thread for executor " + fullId + " interrupted") state = ExecutorState.KILLED killProcess(None) - } - case e: Exception => { + case e: Exception => logError("Error running executor", e) state = ExecutorState.FAILED killProcess(Some(e.toString)) - } } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 1b7637a39c..449beb0811 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -480,7 +480,7 @@ private[deploy] class Worker( memoryUsed += memory_ sendToMaster(ExecutorStateChanged(appId, execId, manager.state, None, None)) } catch { - case e: Exception => { + case e: Exception => logError(s"Failed to launch executor $appId/$execId for ${appDesc.name}.", e) if (executors.contains(appId + "/" + execId)) { executors(appId + "/" + execId).kill() @@ -488,7 +488,6 @@ private[deploy] class Worker( } sendToMaster(ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(e.toString), None)) - } } } @@ -509,7 +508,7 @@ private[deploy] class Worker( } } - case LaunchDriver(driverId, driverDesc) => { + case LaunchDriver(driverId, driverDesc) => logInfo(s"Asked to launch driver $driverId") val driver = new DriverRunner( conf, @@ -525,9 +524,8 @@ private[deploy] class Worker( coresUsed += driverDesc.cores memoryUsed += driverDesc.mem - } - case KillDriver(driverId) => { + case KillDriver(driverId) => logInfo(s"Asked to kill driver $driverId") drivers.get(driverId) match { case Some(runner) => @@ -535,11 +533,9 @@ private[deploy] class Worker( case None => logError(s"Asked to kill unknown driver $driverId") } - } - case driverStateChanged @ DriverStateChanged(driverId, state, exception) => { + case driverStateChanged @ DriverStateChanged(driverId, state, exception) => handleDriverStateChanged(driverStateChanged) - } case ReregisterWithMaster => reregisterWithMaster() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 391eb41190..777020d4d5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -165,12 +165,11 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { } // scalastyle:on classforname } catch { - case e: Exception => { + case e: Exception => totalMb = 2*1024 // scalastyle:off println System.out.println("Failed to get total physical memory. Using " + totalMb + " MB") // scalastyle:on println - } } // Leave out 1 GB for the operating system, but don't return a negative memory size math.max(totalMb - 1024, Utils.DEFAULT_DRIVER_MEM_MB) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 6500cab73b..e75c0cec4a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -107,20 +107,18 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with } val content = - <html> - <body> - {linkToMaster} - <div> - <div style="float:left; margin-right:10px">{backButton}</div> - <div style="float:left;">{range}</div> - <div style="float:right; margin-left:10px">{nextButton}</div> - </div> - <br /> - <div style="height:500px; overflow:auto; padding:5px;"> - <pre>{logText}</pre> - </div> - </body> - </html> + <div> + {linkToMaster} + <div> + <div style="float:left; margin-right:10px">{backButton}</div> + <div style="float:left;">{range}</div> + <div style="float:right; margin-left:10px">{nextButton}</div> + </div> + <br /> + <div style="height:500px; overflow:auto; padding:5px;"> + <pre>{logText}</pre> + </div> + </div> UIUtils.basicSparkPage(content, logType + " log page for " + pageName) } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 320a20033d..71b4ad160d 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -57,16 +57,14 @@ private[spark] class CoarseGrainedExecutorBackend( rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) - ref.ask[RegisterExecutorResponse](RegisterExecutor(executorId, self, cores, extractLogUrls)) + ref.ask[Boolean](RegisterExecutor(executorId, self, cores, extractLogUrls)) }(ThreadUtils.sameThread).onComplete { // This is a very fast action so we can use "ThreadUtils.sameThread" - case Success(msg) => Utils.tryLogNonFatalError { - Option(self).foreach(_.send(msg)) // msg must be RegisterExecutorResponse - } - case Failure(e) => { + case Success(msg) => + // Always receive `true`. Just ignore it + case Failure(e) => logError(s"Cannot register with driver: $driverUrl", e) System.exit(1) - } }(ThreadUtils.sameThread) } @@ -113,9 +111,15 @@ private[spark] class CoarseGrainedExecutorBackend( case Shutdown => stopping.set(true) - executor.stop() - stop() - rpcEnv.shutdown() + new Thread("CoarseGrainedExecutorBackend-stop-executor") { + override def run(): Unit = { + // executor.stop() will call `SparkEnv.stop()` which waits until RpcEnv stops totally. + // However, if `executor.stop()` runs in some thread of RpcEnv, RpcEnv won't be able to + // stop until `executor.stop()` returns, which becomes a dead-lock (See SPARK-14180). + // Therefore, we put this line in a new thread. + executor.stop() + } + }.start() } override def onDisconnected(remoteAddress: RpcAddress): Unit = { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 3201463b8c..9f94fdef24 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -21,6 +21,7 @@ import java.io.{File, NotSerializableException} import java.lang.management.ManagementFactory import java.net.URL import java.nio.ByteBuffer +import java.util.Properties import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConverters._ @@ -206,9 +207,16 @@ private[spark] class Executor( startGCTime = computeTotalGcTime() try { - val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) + val (taskFiles, taskJars, taskProps, taskBytes) = + Task.deserializeWithDependencies(serializedTask) + + // Must be set before updateDependencies() is called, in case fetching dependencies + // requires access to properties contained within (e.g. for access control). + Executor.taskDeserializationProps.set(taskProps) + updateDependencies(taskFiles, taskJars) task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) + task.localProperties = taskProps task.setTaskMemoryManager(taskMemoryManager) // If this task has been killed before we deserialized it, let's quit now. Otherwise, @@ -254,7 +262,7 @@ private[spark] class Executor( if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false) && !threwException) { throw new SparkException(errMsg) } else { - logError(errMsg) + logWarning(errMsg) } } } @@ -321,7 +329,7 @@ private[spark] class Executor( logInfo(s"Executor killed $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) - case cDE: CommitDeniedException => + case CausedBy(cDE: CommitDeniedException) => val reason = cDE.toTaskEndReason execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) @@ -506,3 +514,10 @@ private[spark] class Executor( heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) } } + +private[spark] object Executor { + // This is reserved for internal use by components that need to read task properties before a + // task is fully deserialized. When possible, the TaskContext.getLocalProperty call should be + // used instead. + val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties] +} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala index e07cb31cbe..7153323d01 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala @@ -25,6 +25,6 @@ import org.apache.spark.TaskState.TaskState * A pluggable interface used by the Executor to send updates to the cluster scheduler. */ private[spark] trait ExecutorBackend { - def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) + def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit } diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index 6d30d3c76a..83e11c5e23 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -81,35 +81,9 @@ class InputMetrics private ( */ def readMethod: DataReadMethod.Value = DataReadMethod.withName(_readMethod.localValue) - // Once incBytesRead & intRecordsRead is ready to be removed from the public API - // we can remove the internal versions and make the previous public API private. - // This has been done to suppress warnings when building. - @deprecated("incrementing input metrics is for internal use only", "2.0.0") - def incBytesRead(v: Long): Unit = _bytesRead.add(v) - private[spark] def incBytesReadInternal(v: Long): Unit = _bytesRead.add(v) - @deprecated("incrementing input metrics is for internal use only", "2.0.0") - def incRecordsRead(v: Long): Unit = _recordsRead.add(v) - private[spark] def incRecordsReadInternal(v: Long): Unit = _recordsRead.add(v) + private[spark] def incBytesRead(v: Long): Unit = _bytesRead.add(v) + private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) private[spark] def setBytesRead(v: Long): Unit = _bytesRead.setValue(v) - private[spark] def setReadMethod(v: DataReadMethod.Value): Unit = - _readMethod.setValue(v.toString) + private[spark] def setReadMethod(v: DataReadMethod.Value): Unit = _readMethod.setValue(v.toString) } - -/** - * Deprecated methods to preserve case class matching behavior before Spark 2.0. - */ -object InputMetrics { - - @deprecated("matching on InputMetrics will not be supported in the future", "2.0.0") - def apply(readMethod: DataReadMethod.Value): InputMetrics = { - val im = new InputMetrics - im.setReadMethod(readMethod) - im - } - - @deprecated("matching on InputMetrics will not be supported in the future", "2.0.0") - def unapply(input: InputMetrics): Option[DataReadMethod.Value] = { - Some(input.readMethod) - } -} diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala index 0b37d559c7..93f953846f 100644 --- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -52,18 +52,6 @@ class OutputMetrics private ( } /** - * Create a new [[OutputMetrics]] that is not associated with any particular task. - * - * This is only used for preserving matching behavior on [[OutputMetrics]], which used to be - * a case class before Spark 2.0. Once we remove support for matching on [[OutputMetrics]] - * we can remove this constructor as well. - */ - private[executor] def this() { - this(InternalAccumulator.createOutputAccums() - .map { a => (a.name.get, a) }.toMap[String, Accumulator[_]]) - } - - /** * Total number of bytes written. */ def bytesWritten: Long = _bytesWritten.localValue @@ -84,21 +72,3 @@ class OutputMetrics private ( _writeMethod.setValue(v.toString) } - -/** - * Deprecated methods to preserve case class matching behavior before Spark 2.0. - */ -object OutputMetrics { - - @deprecated("matching on OutputMetrics will not be supported in the future", "2.0.0") - def apply(writeMethod: DataWriteMethod.Value): OutputMetrics = { - val om = new OutputMetrics - om.setWriteMethod(writeMethod) - om - } - - @deprecated("matching on OutputMetrics will not be supported in the future", "2.0.0") - def unapply(output: OutputMetrics): Option[DataWriteMethod.Value] = { - Some(output.writeMethod) - } -} diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 50bb645d97..71a24770b5 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -116,4 +116,25 @@ class ShuffleReadMetrics private ( private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) + /** + * Resets the value of the current metrics (`this`) and and merges all the independent + * [[ShuffleReadMetrics]] into `this`. + */ + private[spark] def setMergeValues(metrics: Seq[ShuffleReadMetrics]): Unit = { + _remoteBlocksFetched.setValue(_remoteBlocksFetched.zero) + _localBlocksFetched.setValue(_localBlocksFetched.zero) + _remoteBytesRead.setValue(_remoteBytesRead.zero) + _localBytesRead.setValue(_localBytesRead.zero) + _fetchWaitTime.setValue(_fetchWaitTime.zero) + _recordsRead.setValue(_recordsRead.zero) + metrics.foreach { metric => + _remoteBlocksFetched.add(metric.remoteBlocksFetched) + _localBlocksFetched.add(metric.localBlocksFetched) + _remoteBytesRead.add(metric.remoteBytesRead) + _localBytesRead.add(metric.localBytesRead) + _fetchWaitTime.add(metric.fetchWaitTime) + _recordsRead.add(metric.recordsRead) + } + } + } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 02219a84ab..bda2a91d9d 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -139,16 +139,6 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se */ def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.localValue - @deprecated("use updatedBlockStatuses instead", "2.0.0") - def updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = { - if (updatedBlockStatuses.nonEmpty) Some(updatedBlockStatuses) else None - } - - @deprecated("setting updated blocks is not allowed", "2.0.0") - def updatedBlocks_=(blocks: Option[Seq[(BlockId, BlockStatus)]]): Unit = { - blocks.foreach(setUpdatedBlockStatuses) - } - // Setters and increment-ers private[spark] def setExecutorDeserializeTime(v: Long): Unit = _executorDeserializeTime.setValue(v) @@ -225,11 +215,6 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se */ def outputMetrics: Option[OutputMetrics] = _outputMetrics - @deprecated("setting OutputMetrics is for internal use only", "2.0.0") - def outputMetrics_=(om: Option[OutputMetrics]): Unit = { - _outputMetrics = om - } - /** * Get or create a new [[OutputMetrics]] associated with this task. */ @@ -285,12 +270,7 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se private[spark] def mergeShuffleReadMetrics(): Unit = synchronized { if (tempShuffleReadMetrics.nonEmpty) { val metrics = new ShuffleReadMetrics(initialAccumsMap) - metrics.setRemoteBlocksFetched(tempShuffleReadMetrics.map(_.remoteBlocksFetched).sum) - metrics.setLocalBlocksFetched(tempShuffleReadMetrics.map(_.localBlocksFetched).sum) - metrics.setFetchWaitTime(tempShuffleReadMetrics.map(_.fetchWaitTime).sum) - metrics.setRemoteBytesRead(tempShuffleReadMetrics.map(_.remoteBytesRead).sum) - metrics.setLocalBytesRead(tempShuffleReadMetrics.map(_.localBytesRead).sum) - metrics.setRecordsRead(tempShuffleReadMetrics.map(_.recordsRead).sum) + metrics.setMergeValues(tempShuffleReadMetrics) _shuffleReadMetrics = Some(metrics) } } @@ -306,11 +286,6 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se */ def shuffleWriteMetrics: Option[ShuffleWriteMetrics] = _shuffleWriteMetrics - @deprecated("setting ShuffleWriteMetrics is for internal use only", "2.0.0") - def shuffleWriteMetrics_=(swm: Option[ShuffleWriteMetrics]): Unit = { - _shuffleWriteMetrics = swm - } - /** * Get or create a new [[ShuffleWriteMetrics]] associated with this task. */ diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala index 770b43697a..5d50e3851a 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala @@ -85,10 +85,12 @@ private[spark] class TypedConfigBuilder[T]( this(parent, converter, Option(_).map(_.toString).orNull) } + /** Apply a transformation to the user-provided values of the config entry. */ def transform(fn: T => T): TypedConfigBuilder[T] = { new TypedConfigBuilder(parent, s => fn(converter(s)), stringConverter) } + /** Check that user-provided values for the config match a pre-defined set. */ def checkValues(validValues: Set[T]): TypedConfigBuilder[T] = { transform { v => if (!validValues.contains(v)) { @@ -99,30 +101,38 @@ private[spark] class TypedConfigBuilder[T]( } } + /** Turns the config entry into a sequence of values of the underlying type. */ def toSequence: TypedConfigBuilder[Seq[T]] = { new TypedConfigBuilder(parent, stringToSeq(_, converter), seqToString(_, stringConverter)) } - /** Creates a [[ConfigEntry]] that does not require a default value. */ - def optional: OptionalConfigEntry[T] = { - new OptionalConfigEntry[T](parent.key, converter, stringConverter, parent._doc, parent._public) + /** Creates a [[ConfigEntry]] that does not have a default value. */ + def createOptional: OptionalConfigEntry[T] = { + val entry = new OptionalConfigEntry[T](parent.key, converter, stringConverter, parent._doc, + parent._public) + parent._onCreate.foreach(_(entry)) + entry } /** Creates a [[ConfigEntry]] that has a default value. */ - def withDefault(default: T): ConfigEntry[T] = { + def createWithDefault(default: T): ConfigEntry[T] = { val transformedDefault = converter(stringConverter(default)) - new ConfigEntryWithDefault[T](parent.key, transformedDefault, converter, stringConverter, - parent._doc, parent._public) + val entry = new ConfigEntryWithDefault[T](parent.key, transformedDefault, converter, + stringConverter, parent._doc, parent._public) + parent._onCreate.foreach(_(entry)) + entry } /** * Creates a [[ConfigEntry]] that has a default value. The default value is provided as a * [[String]] and must be a valid value for the entry. */ - def withDefaultString(default: String): ConfigEntry[T] = { + def createWithDefaultString(default: String): ConfigEntry[T] = { val typedDefault = converter(default) - new ConfigEntryWithDefault[T](parent.key, typedDefault, converter, stringConverter, parent._doc, - parent._public) + val entry = new ConfigEntryWithDefault[T](parent.key, typedDefault, converter, stringConverter, + parent._doc, parent._public) + parent._onCreate.foreach(_(entry)) + entry } } @@ -136,10 +146,11 @@ private[spark] case class ConfigBuilder(key: String) { import ConfigHelpers._ - var _public = true - var _doc = "" + private[config] var _public = true + private[config] var _doc = "" + private[config] var _onCreate: Option[ConfigEntry[_] => Unit] = None - def internal: ConfigBuilder = { + def internal(): ConfigBuilder = { _public = false this } @@ -149,6 +160,15 @@ private[spark] case class ConfigBuilder(key: String) { this } + /** + * Registers a callback for when the config entry is finally instantiated. Currently used by + * SQLConf to keep track of SQL configuration entries. + */ + def onCreate(callback: ConfigEntry[_] => Unit): ConfigBuilder = { + _onCreate = Option(callback) + this + } + def intConf: TypedConfigBuilder[Int] = { new TypedConfigBuilder(this, toNumber(_, _.toInt, key, "int")) } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index f2f20b3207..94b50ee065 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -18,59 +18,75 @@ package org.apache.spark.internal import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.network.util.ByteUnit package object config { private[spark] val DRIVER_CLASS_PATH = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH).stringConf.optional + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH).stringConf.createOptional private[spark] val DRIVER_JAVA_OPTIONS = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS).stringConf.optional + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS).stringConf.createOptional private[spark] val DRIVER_LIBRARY_PATH = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH).stringConf.optional + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH).stringConf.createOptional private[spark] val DRIVER_USER_CLASS_PATH_FIRST = - ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.withDefault(false) + ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.createWithDefault(false) + + private[spark] val DRIVER_MEMORY = ConfigBuilder("spark.driver.memory") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("1g") private[spark] val EXECUTOR_CLASS_PATH = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.optional + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional private[spark] val EXECUTOR_JAVA_OPTIONS = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS).stringConf.optional + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS).stringConf.createOptional private[spark] val EXECUTOR_LIBRARY_PATH = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH).stringConf.optional + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH).stringConf.createOptional private[spark] val EXECUTOR_USER_CLASS_PATH_FIRST = - ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.withDefault(false) + ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.createWithDefault(false) + + private[spark] val EXECUTOR_MEMORY = ConfigBuilder("spark.executor.memory") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("1g") - private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal - .booleanConf.withDefault(false) + private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() + .booleanConf.createWithDefault(false) - private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.withDefault(1) + private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.createWithDefault(1) private[spark] val DYN_ALLOCATION_MIN_EXECUTORS = - ConfigBuilder("spark.dynamicAllocation.minExecutors").intConf.withDefault(0) + ConfigBuilder("spark.dynamicAllocation.minExecutors").intConf.createWithDefault(0) private[spark] val DYN_ALLOCATION_INITIAL_EXECUTORS = ConfigBuilder("spark.dynamicAllocation.initialExecutors") .fallbackConf(DYN_ALLOCATION_MIN_EXECUTORS) private[spark] val DYN_ALLOCATION_MAX_EXECUTORS = - ConfigBuilder("spark.dynamicAllocation.maxExecutors").intConf.withDefault(Int.MaxValue) + ConfigBuilder("spark.dynamicAllocation.maxExecutors").intConf.createWithDefault(Int.MaxValue) private[spark] val SHUFFLE_SERVICE_ENABLED = - ConfigBuilder("spark.shuffle.service.enabled").booleanConf.withDefault(false) + ConfigBuilder("spark.shuffle.service.enabled").booleanConf.createWithDefault(false) private[spark] val KEYTAB = ConfigBuilder("spark.yarn.keytab") .doc("Location of user's keytab.") - .stringConf.optional + .stringConf.createOptional private[spark] val PRINCIPAL = ConfigBuilder("spark.yarn.principal") .doc("Name of the Kerberos principal.") - .stringConf.optional + .stringConf.createOptional - private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances").intConf.optional + private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances") + .intConf + .createOptional + private[spark] val PY_FILES = ConfigBuilder("spark.submit.pyFiles") + .internal() + .stringConf + .toSequence + .createWithDefault(Nil) } diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala index a2add61617..31b9c5edf0 100644 --- a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala +++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala @@ -37,7 +37,6 @@ private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, comm override def buildCommand(env: JMap[String, String]): JList[String] = { val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator)) - cmd.add(s"-Xms${memoryMb}M") cmd.add(s"-Xmx${memoryMb}M") command.javaOpts.foreach(cmd.add) CommandBuilderUtils.addPermGenSizeOpt(cmd) diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 891facba33..607283a306 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -33,11 +33,8 @@ object SparkHadoopMapRedUtil extends Logging { * the driver in order to determine whether this attempt can commit (please see SPARK-4879 for * details). * - * Output commit coordinator is only contacted when the following two configurations are both set - * to `true`: - * - * - `spark.speculation` - * - `spark.hadoop.outputCommitCoordination.enabled` + * Output commit coordinator is only used when `spark.hadoop.outputCommitCoordination.enabled` + * is set to true (which is the default). */ def commitTask( committer: MapReduceOutputCommitter, @@ -64,11 +61,10 @@ object SparkHadoopMapRedUtil extends Logging { if (committer.needsTaskCommit(mrTaskContext)) { val shouldCoordinateWithDriver: Boolean = { val sparkConf = SparkEnv.get.conf - // We only need to coordinate with the driver if there are multiple concurrent task - // attempts, which should only occur if speculation is enabled - val speculationEnabled = sparkConf.getBoolean("spark.speculation", defaultValue = false) - // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs - sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", speculationEnabled) + // We only need to coordinate with the driver if there are concurrent task attempts. + // Note that this could happen even when speculation is not enabled (e.g. see SPARK-8029). + // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs. + sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", defaultValue = true) } if (shouldCoordinateWithDriver) { diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 10656bc8c8..0210217e41 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -23,6 +23,7 @@ import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.storage.BlockId import org.apache.spark.storage.memory.MemoryStore +import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.memory.MemoryAllocator @@ -190,6 +191,8 @@ private[spark] abstract class MemoryManager( if (conf.getBoolean("spark.memory.offHeap.enabled", false)) { require(conf.getSizeAsBytes("spark.memory.offHeap.size", 0) > 0, "spark.memory.offHeap.size must be > 0 when spark.memory.offHeap.enabled == true") + require(Platform.unaligned(), + "No support for unaligned Unsafe. Set spark.memory.offHeap.enabled to false.") MemoryMode.OFF_HEAP } else { MemoryMode.ON_HEAP diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index a67e8da26b..0b552cabfc 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -35,6 +35,11 @@ private[memory] class StorageMemoryPool( memoryMode: MemoryMode ) extends MemoryPool(lock) with Logging { + private[this] val poolName: String = memoryMode match { + case MemoryMode.ON_HEAP => "on-heap storage" + case MemoryMode.OFF_HEAP => "off-heap storage" + } + @GuardedBy("lock") private[this] var _memoryUsed: Long = 0L @@ -60,7 +65,7 @@ private[memory] class StorageMemoryPool( /** * Acquire N bytes of memory to cache the given block, evicting existing ones if necessary. - * + * * @return whether all N bytes were successfully granted. */ def acquireMemory(blockId: BlockId, numBytes: Long): Boolean = lock.synchronized { @@ -83,9 +88,8 @@ private[memory] class StorageMemoryPool( assert(numBytesToAcquire >= 0) assert(numBytesToFree >= 0) assert(memoryUsed <= poolSize) - // Once we support off-heap caching, this will need to change: - if (numBytesToFree > 0 && memoryMode == MemoryMode.ON_HEAP) { - memoryStore.evictBlocksToFreeSpace(Some(blockId), numBytesToFree) + if (numBytesToFree > 0) { + memoryStore.evictBlocksToFreeSpace(Some(blockId), numBytesToFree, memoryMode) } // NOTE: If the memory store evicts blocks, then those evictions will synchronously call // back into this StorageMemoryPool in order to free memory. Therefore, these variables @@ -122,14 +126,8 @@ private[memory] class StorageMemoryPool( val remainingSpaceToFree = spaceToFree - spaceFreedByReleasingUnusedMemory if (remainingSpaceToFree > 0) { // If reclaiming free memory did not adequately shrink the pool, begin evicting blocks: - val spaceFreedByEviction = { - // Once we support off-heap caching, this will need to change: - if (memoryMode == MemoryMode.ON_HEAP) { - memoryStore.evictBlocksToFreeSpace(None, remainingSpaceToFree) - } else { - 0 - } - } + val spaceFreedByEviction = + memoryStore.evictBlocksToFreeSpace(None, remainingSpaceToFree, memoryMode) // When a block is released, BlockManager.dropFromMemory() calls releaseMemory(), so we do // not need to decrement _memoryUsed here. However, we do need to decrement the pool size. decrementPoolSize(spaceFreedByEviction) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 4da1017d28..0fed991049 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -196,10 +196,9 @@ private[spark] class MetricsSystem private ( sinks += sink.asInstanceOf[Sink] } } catch { - case e: Exception => { + case e: Exception => logError("Sink class " + classPath + " cannot be instantiated") throw e - } } } } diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index e43e3a2de2..09ce012e4e 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -36,7 +36,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch * local blocks or put local blocks. */ - def init(blockDataManager: BlockDataManager) + def init(blockDataManager: BlockDataManager): Unit /** * Tear down the transfer service. diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 5f3d4532dd..33a3219607 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -39,7 +39,11 @@ import org.apache.spark.util.Utils /** * A BlockTransferService that uses Netty to fetch a set of blocks at at time. */ -class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManager, numCores: Int) +private[spark] class NettyBlockTransferService( + conf: SparkConf, + securityManager: SecurityManager, + override val hostName: String, + numCores: Int) extends BlockTransferService { // TODO: Don't use Java serialization, use a more cross-version compatible serialization format. @@ -65,13 +69,13 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage clientFactory = transportContext.createClientFactory(clientBootstrap.toSeq.asJava) server = createServer(serverBootstrap.toList) appId = conf.getAppId - logInfo("Server created on " + server.getPort) + logInfo(s"Server created on ${hostName}:${server.getPort}") } /** Creates and binds the TransportServer, possibly trying multiple ports. */ private def createServer(bootstraps: List[TransportServerBootstrap]): TransportServer = { def startService(port: Int): (TransportServer, Int) = { - val server = transportContext.createServer(port, bootstraps.asJava) + val server = transportContext.createServer(hostName, port, bootstraps.asJava) (server, server.getPort) } @@ -109,8 +113,6 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage } } - override def hostName: String = Utils.localHostName() - override def port: Int = server.getPort override def uploadBlock( diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index 48b9434153..ab6aba6fc7 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -21,5 +21,22 @@ package org.apache.spark.partial * A Double value with error bars and associated confidence. */ class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { + override def toString(): String = "[%.3f, %.3f]".format(low, high) + + override def hashCode: Int = + this.mean.hashCode ^ this.confidence.hashCode ^ this.low.hashCode ^ this.high.hashCode + + /** + * Note that consistent with Double, any NaN value will make equality false + */ + override def equals(that: Any): Boolean = + that match { + case that: BoundedDouble => + this.mean == that.mean && + this.confidence == that.confidence && + this.low == that.low && + this.high == that.high + case _ => false + } } diff --git a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala index 44295e5a1a..5fe3358316 100644 --- a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala @@ -29,8 +29,9 @@ import org.apache.spark.util.StatCounter private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[StatCounter, BoundedDouble] { + // modified in merge var outputsMerged = 0 - var counter = new StatCounter + val counter = new StatCounter override def merge(outputId: Int, taskResult: StatCounter) { outputsMerged += 1 @@ -40,30 +41,39 @@ private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double) override def currentResult(): BoundedDouble = { if (outputsMerged == totalOutputs) { new BoundedDouble(counter.sum, 1.0, counter.sum, counter.sum) - } else if (outputsMerged == 0) { + } else if (outputsMerged == 0 || counter.count == 0) { new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) } else { val p = outputsMerged.toDouble / totalOutputs val meanEstimate = counter.mean - val meanVar = counter.sampleVariance / counter.count val countEstimate = (counter.count + 1 - p) / p - val countVar = (counter.count + 1) * (1 - p) / (p * p) val sumEstimate = meanEstimate * countEstimate - val sumVar = (meanEstimate * meanEstimate * countVar) + - (countEstimate * countEstimate * meanVar) + - (meanVar * countVar) - val sumStdev = math.sqrt(sumVar) - val confFactor = { - if (counter.count > 100) { + + val meanVar = counter.sampleVariance / counter.count + + // branch at this point because counter.count == 1 implies counter.sampleVariance == Nan + // and we don't want to ever return a bound of NaN + if (meanVar.isNaN || counter.count == 1) { + new BoundedDouble(sumEstimate, confidence, Double.NegativeInfinity, Double.PositiveInfinity) + } else { + val countVar = (counter.count + 1) * (1 - p) / (p * p) + val sumVar = (meanEstimate * meanEstimate * countVar) + + (countEstimate * countEstimate * meanVar) + + (meanVar * countVar) + val sumStdev = math.sqrt(sumVar) + val confFactor = if (counter.count > 100) { new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) } else { + // note that if this goes to 0, TDistribution will throw an exception. + // Hence special casing 1 above. val degreesOfFreedom = (counter.count - 1).toInt new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2) } + + val low = sumEstimate - confFactor * sumStdev + val high = sumEstimate + confFactor * sumStdev + new BoundedDouble(sumEstimate, confidence, low, high) } - val low = sumEstimate - confFactor * sumStdev - val high = sumEstimate + confFactor * sumStdev - new BoundedDouble(sumEstimate, confidence, low, high) } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index 8358244987..63d1d1767a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -35,9 +35,9 @@ class BlockRDD[T: ClassTag](sc: SparkContext, @transient val blockIds: Array[Blo override def getPartitions: Array[Partition] = { assertValid() - (0 until blockIds.length).map(i => { + (0 until blockIds.length).map { i => new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] - }).toArray + }.toArray } override def compute(split: Partition, context: TaskContext): Iterator[T] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index e5ebc63082..7bc1eb0436 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -29,10 +29,12 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.util.collection.{CompactBuffer, ExternalAppendOnlyMap} import org.apache.spark.util.Utils -/** The references to rdd and splitIndex are transient because redundant information is stored - * in the CoGroupedRDD object. Because CoGroupedRDD is serialized separately from - * CoGroupPartition, if rdd and splitIndex aren't transient, they'll be included twice in the - * task closure. */ +/** + * The references to rdd and splitIndex are transient because redundant information is stored + * in the CoGroupedRDD object. Because CoGroupedRDD is serialized separately from + * CoGroupPartition, if rdd and splitIndex aren't transient, they'll be included twice in the + * task closure. + */ private[spark] case class NarrowCoGroupSplitDep( @transient rdd: RDD[_], @transient splitIndex: Int, diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 5e9230e733..368916a39e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -166,8 +166,8 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { val counters = new Array[Long](buckets.length - 1) while (iter.hasNext) { bucketFunction(iter.next()) match { - case Some(x: Int) => {counters(x) += 1} - case _ => {} + case Some(x: Int) => counters(x) += 1 + case _ => // No-Op } } Iterator(counters) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 08db96edd6..35d190b464 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -213,15 +213,13 @@ class HadoopRDD[K, V]( logInfo("Input split: " + split.inputSplit) val jobConf = getJobConf() - // TODO: there is a lot of duplicate code between this and NewHadoopRDD and SqlNewHadoopRDD - val inputMetrics = context.taskMetrics().registerInputMetrics(DataReadMethod.Hadoop) val existingBytesRead = inputMetrics.bytesRead // Sets the thread local variable for the file's name split.inputSplit.value match { - case fs: FileSplit => SqlNewHadoopRDDState.setInputFileName(fs.getPath.toString) - case _ => SqlNewHadoopRDDState.unsetInputFileName() + case fs: FileSplit => InputFileNameHolder.setInputFileName(fs.getPath.toString) + case _ => InputFileNameHolder.unsetInputFileName() } // Find a function that will return the FileSystem bytes read by this thread. Do this before @@ -261,7 +259,7 @@ class HadoopRDD[K, V]( finished = true } if (!finished) { - inputMetrics.incRecordsReadInternal(1) + inputMetrics.incRecordsRead(1) } if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { updateBytesRead() @@ -271,7 +269,7 @@ class HadoopRDD[K, V]( override def close() { if (reader != null) { - SqlNewHadoopRDDState.unsetInputFileName() + InputFileNameHolder.unsetInputFileName() // Close the reader and release it. Note: it's very important that we don't close the // reader more than once, since that exposes us to MAPREDUCE-5918 when running against // Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to non-deterministic @@ -293,7 +291,7 @@ class HadoopRDD[K, V]( // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.incBytesReadInternal(split.inputSplit.value.getLength) + inputMetrics.incBytesRead(split.inputSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) @@ -424,7 +422,7 @@ private[spark] object HadoopRDD extends Logging { private[spark] def convertSplitLocationInfo(infos: Array[AnyRef]): Seq[String] = { val out = ListBuffer[String]() - infos.foreach { loc => { + infos.foreach { loc => val locationStr = HadoopRDD.SPLIT_INFO_REFLECTIONS.get. getLocation.invoke(loc).asInstanceOf[String] if (locationStr != "localhost") { @@ -436,7 +434,7 @@ private[spark] object HadoopRDD extends Logging { out += new HostTaskLocation(locationStr).toString } } - }} + } out.seq } } diff --git a/core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDDState.scala b/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala index 3f15fff793..108e9d2558 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDDState.scala +++ b/core/src/main/scala/org/apache/spark/rdd/InputFileNameHolder.scala @@ -20,10 +20,10 @@ package org.apache.spark.rdd import org.apache.spark.unsafe.types.UTF8String /** - * State for SqlNewHadoopRDD objects. This is split this way because of the package splits. - * TODO: Move/Combine this with org.apache.spark.sql.datasources.SqlNewHadoopRDD + * This holds file names of the current Spark task. This is used in HadoopRDD, + * FileScanRDD and InputFileName function in Spark SQL. */ -private[spark] object SqlNewHadoopRDDState { +private[spark] object InputFileNameHolder { /** * The thread variable for the name of the current file being read. This is used by * the InputFileName function in Spark SQL. diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index fb9606ae38..3ccd616cbf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -189,7 +189,7 @@ class NewHadoopRDD[K, V]( } havePair = false if (!finished) { - inputMetrics.incRecordsReadInternal(1) + inputMetrics.incRecordsRead(1) } if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { updateBytesRead() @@ -220,7 +220,7 @@ class NewHadoopRDD[K, V]( // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.incBytesReadInternal(split.serializableHadoopSplit.value.getLength) + inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 363004e587..a5992022d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -86,12 +86,11 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, def inRange(k: K): Boolean = ordering.gteq(k, lower) && ordering.lteq(k, upper) val rddToFilter: RDD[P] = self.partitioner match { - case Some(rp: RangePartitioner[K, V]) => { + case Some(rp: RangePartitioner[K, V]) => val partitionIndicies = (rp.getPartition(lower), rp.getPartition(upper)) match { case (l, u) => Math.min(l, u) to Math.max(l, u) } PartitionPruningRDD.create(self, partitionIndicies.contains) - } case _ => self } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 296179b75b..085829af6e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -1111,9 +1111,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) maybeUpdateOutputMetrics(outputMetricsAndBytesWrittenCallback, recordsWritten) recordsWritten += 1 } - } { - writer.close(hadoopContext) - } + }(finallyBlock = writer.close(hadoopContext)) committer.commitTask(hadoopContext) outputMetricsAndBytesWrittenCallback.foreach { case (om, callback) => om.setBytesWritten(callback()) @@ -1200,9 +1198,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) maybeUpdateOutputMetrics(outputMetricsAndBytesWrittenCallback, recordsWritten) recordsWritten += 1 } - } { - writer.close() - } + }(finallyBlock = writer.close()) writer.commit() outputMetricsAndBytesWrittenCallback.foreach { case (om, callback) => om.setBytesWritten(callback()) diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 582fa93afe..bb84e4af15 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -121,14 +121,14 @@ private object ParallelCollectionRDD { // Sequences need to be sliced at the same set of index positions for operations // like RDD.zip() to behave as expected def positions(length: Long, numSlices: Int): Iterator[(Int, Int)] = { - (0 until numSlices).iterator.map(i => { + (0 until numSlices).iterator.map { i => val start = ((i * length) / numSlices).toInt val end = (((i + 1) * length) / numSlices).toInt (start, end) - }) + } } seq match { - case r: Range => { + case r: Range => positions(r.length, numSlices).zipWithIndex.map({ case ((start, end), index) => // If the range is inclusive, use inclusive range for the last slice if (r.isInclusive && index == numSlices - 1) { @@ -138,8 +138,7 @@ private object ParallelCollectionRDD { new Range(r.start + start * r.step, r.start + end * r.step, r.step) } }).toSeq.asInstanceOf[Seq[Seq[T]]] - } - case nr: NumericRange[_] => { + case nr: NumericRange[_] => // For ranges of Long, Double, BigInteger, etc val slices = new ArrayBuffer[Seq[T]](numSlices) var r = nr @@ -149,14 +148,12 @@ private object ParallelCollectionRDD { r = r.drop(sliceSize) } slices - } - case _ => { + case _ => val array = seq.toArray // To prevent O(n^2) operations for List etc positions(array.length, numSlices).map({ case (start, end) => array.slice(start, end).toSeq }).toSeq - } } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index 9e3880714a..0abba15bec 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -68,9 +68,9 @@ class PartitionerAwareUnionRDD[T: ClassTag]( override def getPartitions: Array[Partition] = { val numPartitions = partitioner.get.numPartitions - (0 until numPartitions).map(index => { + (0 until numPartitions).map { index => new PartitionerAwareUnionRDDPartition(rdds, index) - }).toArray + }.toArray } // Get the location where most of the partitions of parent RDDs are located @@ -78,11 +78,10 @@ class PartitionerAwareUnionRDD[T: ClassTag]( logDebug("Finding preferred location for " + this + ", partition " + s.index) val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents val locations = rdds.zip(parentPartitions).flatMap { - case (rdd, part) => { + case (rdd, part) => val parentLocations = currPrefLocs(rdd, part) logDebug("Location of " + rdd + " partition " + part.index + " = " + parentLocations) parentLocations - } } val location = if (locations.isEmpty) { None diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index f96551c793..36ff3bcaae 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -255,8 +255,8 @@ abstract class RDD[T: ClassTag]( } /** - * Returns the number of partitions of this RDD. - */ + * Returns the number of partitions of this RDD. + */ @Since("1.6.0") final def getNumPartitions: Int = partitions.length @@ -333,10 +333,10 @@ abstract class RDD[T: ClassTag]( case Left(blockResult) => if (readCachedBlock) { val existingMetrics = context.taskMetrics().registerInputMetrics(blockResult.readMethod) - existingMetrics.incBytesReadInternal(blockResult.bytes) + existingMetrics.incBytesRead(blockResult.bytes) new InterruptibleIterator[T](context, blockResult.data.asInstanceOf[Iterator[T]]) { override def next(): T = { - existingMetrics.incRecordsReadInternal(1) + existingMetrics.incRecordsRead(1) delegate.next() } } @@ -568,11 +568,7 @@ abstract class RDD[T: ClassTag]( * times (use `.distinct()` to eliminate them). */ def union(other: RDD[T]): RDD[T] = withScope { - if (partitioner.isDefined && other.partitioner == partitioner) { - new PartitionerAwareUnionRDD(sc, Array(this, other)) - } else { - new UnionRDD(sc, Array(this, other)) - } + sc.union(this, other) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5cdc91316b..c27aad268d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -950,13 +950,6 @@ class DAGScheduler( // First figure out the indexes of partition ids to compute. val partitionsToCompute: Seq[Int] = stage.findMissingPartitions() - // Create internal accumulators if the stage has no accumulators initialized. - // Reset internal accumulators only if this stage is not partially submitted - // Otherwise, we may override existing accumulator values from some tasks - if (stage.internalAccumulators.isEmpty || stage.numPartitions == partitionsToCompute.size) { - stage.resetInternalAccumulators() - } - // Use the scheduling pool, job group, description, etc. from an ActiveJob associated // with this Stage val properties = jobIdToActiveJob(jobId).properties @@ -1036,7 +1029,7 @@ class DAGScheduler( val locs = taskIdToLocations(id) val part = stage.rdd.partitions(id) new ShuffleMapTask(stage.id, stage.latestInfo.attemptId, - taskBinary, part, locs, stage.internalAccumulators) + taskBinary, part, locs, stage.latestInfo.internalAccumulators, properties) } case stage: ResultStage => @@ -1046,7 +1039,7 @@ class DAGScheduler( val part = stage.rdd.partitions(p) val locs = taskIdToLocations(id) new ResultTask(stage.id, stage.latestInfo.attemptId, - taskBinary, part, locs, id, stage.internalAccumulators) + taskBinary, part, locs, id, properties, stage.latestInfo.internalAccumulators) } } } catch { diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 0640f26051..a6b032cc00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -57,11 +57,10 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // Since we are not doing canonicalization of path, this can be wrong : like relative vs // absolute path .. which is fine, this is best case effort to remove duplicates - right ? override def equals(other: Any): Boolean = other match { - case that: InputFormatInfo => { + case that: InputFormatInfo => // not checking config - that should be fine, right ? this.inputFormatClazz == that.inputFormatClazz && this.path == that.path - } case _ => false } @@ -86,10 +85,9 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl } } catch { - case e: ClassNotFoundException => { + case e: ClassNotFoundException => throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + " cannot be found ?", e) - } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala index 50c2b9acd6..e0f7c8f021 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala @@ -23,6 +23,6 @@ package org.apache.spark.scheduler * job fails (and no further taskSucceeded events will happen). */ private[spark] trait JobListener { - def taskSucceeded(index: Int, result: Any) - def jobFailed(exception: Exception) + def taskSucceeded(index: Int, result: Any): Unit + def jobFailed(exception: Exception): Unit } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index cd2736e196..db6276f75d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import java.io._ import java.nio.ByteBuffer +import java.util.Properties import org.apache.spark._ import org.apache.spark.broadcast.Broadcast @@ -38,6 +39,7 @@ import org.apache.spark.rdd.RDD * @param locs preferred task execution locations for locality scheduling * @param outputId index of the task in this job (a job can launch tasks on only a subset of the * input RDD's partitions). + * @param localProperties copy of thread-local properties set by the user on the driver side. * @param _initialAccums initial set of accumulators to be used in this task for tracking * internal metrics. Other accumulators will be registered later when * they are deserialized on the executors. @@ -49,8 +51,9 @@ private[spark] class ResultTask[T, U]( partition: Partition, locs: Seq[TaskLocation], val outputId: Int, + localProperties: Properties, _initialAccums: Seq[Accumulator[_]] = InternalAccumulator.createAll()) - extends Task[U](stageId, stageAttemptId, partition.index, _initialAccums) + extends Task[U](stageId, stageAttemptId, partition.index, _initialAccums, localProperties) with Serializable { @transient private[this] val preferredLocs: Seq[TaskLocation] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 5baebe8c1f..100ed76ecb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -34,9 +34,9 @@ import org.apache.spark.util.Utils private[spark] trait SchedulableBuilder { def rootPool: Pool - def buildPools() + def buildPools(): Unit - def addTaskSetManager(manager: Schedulable, properties: Properties) + def addTaskSetManager(manager: Schedulable, properties: Properties): Unit } private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index e30964a01b..b7cab7013e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer +import java.util.Properties import scala.language.existentials @@ -42,6 +43,7 @@ import org.apache.spark.shuffle.ShuffleWriter * @param _initialAccums initial set of accumulators to be used in this task for tracking * internal metrics. Other accumulators will be registered later when * they are deserialized on the executors. + * @param localProperties copy of thread-local properties set by the user on the driver side. */ private[spark] class ShuffleMapTask( stageId: Int, @@ -49,13 +51,14 @@ private[spark] class ShuffleMapTask( taskBinary: Broadcast[Array[Byte]], partition: Partition, @transient private var locs: Seq[TaskLocation], - _initialAccums: Seq[Accumulator[_]]) - extends Task[MapStatus](stageId, stageAttemptId, partition.index, _initialAccums) + _initialAccums: Seq[Accumulator[_]], + localProperties: Properties) + extends Task[MapStatus](stageId, stageAttemptId, partition.index, _initialAccums, localProperties) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ def this(partitionId: Int) { - this(0, 0, null, new Partition { override def index: Int = 0 }, null, null) + this(0, 0, null, new Partition { override def index: Int = 0 }, null, null, new Properties) } @transient private val preferredLocs: Seq[TaskLocation] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 586173f180..080ea6c33a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -151,275 +151,152 @@ private[spark] trait SparkHistoryListenerFactory { def createListeners(conf: SparkConf, sparkUI: SparkUI): Seq[SparkListener] } + /** - * :: DeveloperApi :: - * Interface for listening to events from the Spark scheduler. Note that this is an internal - * interface which might change in different Spark releases. Java clients should extend - * {@link JavaSparkListener} + * Interface for listening to events from the Spark scheduler. Most applications should probably + * extend SparkListener or SparkFirehoseListener directly, rather than implementing this class. + * + * Note that this is an internal interface which might change in different Spark releases. */ -@DeveloperApi -trait SparkListener { +private[spark] trait SparkListenerInterface { + /** * Called when a stage completes successfully or fails, with information on the completed stage. */ - def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { } + def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit /** * Called when a stage is submitted */ - def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } + def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit /** * Called when a task starts */ - def onTaskStart(taskStart: SparkListenerTaskStart) { } + def onTaskStart(taskStart: SparkListenerTaskStart): Unit /** * Called when a task begins remotely fetching its result (will not be called for tasks that do * not need to fetch the result remotely). */ - def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { } + def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult): Unit /** * Called when a task ends */ - def onTaskEnd(taskEnd: SparkListenerTaskEnd) { } + def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit /** * Called when a job starts */ - def onJobStart(jobStart: SparkListenerJobStart) { } + def onJobStart(jobStart: SparkListenerJobStart): Unit /** * Called when a job ends */ - def onJobEnd(jobEnd: SparkListenerJobEnd) { } + def onJobEnd(jobEnd: SparkListenerJobEnd): Unit /** * Called when environment properties have been updated */ - def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { } + def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit /** * Called when a new block manager has joined */ - def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { } + def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit /** * Called when an existing block manager has been removed */ - def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { } + def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit /** * Called when an RDD is manually unpersisted by the application */ - def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } + def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit /** * Called when the application starts */ - def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } + def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit /** * Called when the application ends */ - def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } + def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit /** * Called when the driver receives task metrics from an executor in a heartbeat. */ - def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { } + def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit /** * Called when the driver registers a new executor. */ - def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) { } + def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit /** * Called when the driver removes an executor. */ - def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved) { } + def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit /** * Called when the driver receives a block update info. */ - def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated) { } + def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit /** * Called when other events like SQL-specific events are posted. */ - def onOtherEvent(event: SparkListenerEvent) { } + def onOtherEvent(event: SparkListenerEvent): Unit } + /** * :: DeveloperApi :: - * Simple SparkListener that logs a few summary statistics when each stage completes + * A default implementation for [[SparkListenerInterface]] that has no-op implementations for + * all callbacks. + * + * Note that this is an internal interface which might change in different Spark releases. */ @DeveloperApi -class StatsReportListener extends SparkListener with Logging { - - import org.apache.spark.scheduler.StatsReportListener._ - - private val taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val info = taskEnd.taskInfo - val metrics = taskEnd.taskMetrics - if (info != null && metrics != null) { - taskInfoMetrics += ((info, metrics)) - } - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - implicit val sc = stageCompleted - this.logInfo(s"Finished stage: ${getStatusDetail(stageCompleted.stageInfo)}") - showMillisDistribution("task runtime:", (info, _) => Some(info.duration), taskInfoMetrics) - - // Shuffle write - showBytesDistribution("shuffle bytes written:", - (_, metric) => metric.shuffleWriteMetrics.map(_.bytesWritten), taskInfoMetrics) - - // Fetch & I/O - showMillisDistribution("fetch wait time:", - (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime), taskInfoMetrics) - showBytesDistribution("remote bytes read:", - (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead), taskInfoMetrics) - showBytesDistribution("task result size:", - (_, metric) => Some(metric.resultSize), taskInfoMetrics) - - // Runtime breakdown - val runtimePcts = taskInfoMetrics.map { case (info, metrics) => - RuntimePercentage(info.duration, metrics) - } - showDistribution("executor (non-fetch) time pct: ", - Distribution(runtimePcts.map(_.executorPct * 100)), "%2.0f %%") - showDistribution("fetch wait time pct: ", - Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%") - showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%") - taskInfoMetrics.clear() - } - - private def getStatusDetail(info: StageInfo): String = { - val failureReason = info.failureReason.map("(" + _ + ")").getOrElse("") - val timeTaken = info.submissionTime.map( - x => info.completionTime.getOrElse(System.currentTimeMillis()) - x - ).getOrElse("-") - - s"Stage(${info.stageId}, ${info.attemptId}); Name: '${info.name}'; " + - s"Status: ${info.getStatusString}$failureReason; numTasks: ${info.numTasks}; " + - s"Took: $timeTaken msec" - } +abstract class SparkListener extends SparkListenerInterface { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { } -} + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { } -private[spark] object StatsReportListener extends Logging { - - // For profiling, the extremes are more interesting - val percentiles = Array[Int](0, 5, 10, 25, 50, 75, 90, 95, 100) - val probabilities = percentiles.map(_ / 100.0) - val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" - - def extractDoubleDistribution( - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], - getMetric: (TaskInfo, TaskMetrics) => Option[Double]): Option[Distribution] = { - Distribution(taskInfoMetrics.flatMap { case (info, metric) => getMetric(info, metric) }) - } - - // Is there some way to setup the types that I can get rid of this completely? - def extractLongDistribution( - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], - getMetric: (TaskInfo, TaskMetrics) => Option[Long]): Option[Distribution] = { - extractDoubleDistribution( - taskInfoMetrics, - (info, metric) => { getMetric(info, metric).map(_.toDouble) }) - } - - def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { - val stats = d.statCounter - val quantiles = d.getQuantiles(probabilities).map(formatNumber) - logInfo(heading + stats) - logInfo(percentilesHeader) - logInfo("\t" + quantiles.mkString("\t")) - } - - def showDistribution( - heading: String, - dOpt: Option[Distribution], - formatNumber: Double => String) { - dOpt.foreach { d => showDistribution(heading, d, formatNumber)} - } - - def showDistribution(heading: String, dOpt: Option[Distribution], format: String) { - def f(d: Double): String = format.format(d) - showDistribution(heading, dOpt, f _) - } - - def showDistribution( - heading: String, - format: String, - getMetric: (TaskInfo, TaskMetrics) => Option[Double], - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { - showDistribution(heading, extractDoubleDistribution(taskInfoMetrics, getMetric), format) - } - - def showBytesDistribution( - heading: String, - getMetric: (TaskInfo, TaskMetrics) => Option[Long], - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { - showBytesDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) - } - - def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { - dOpt.foreach { dist => showBytesDistribution(heading, dist) } - } - - def showBytesDistribution(heading: String, dist: Distribution) { - showDistribution(heading, dist, (d => Utils.bytesToString(d.toLong)): Double => String) - } - - def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { - showDistribution(heading, dOpt, - (d => StatsReportListener.millisToString(d.toLong)): Double => String) - } - - def showMillisDistribution( - heading: String, - getMetric: (TaskInfo, TaskMetrics) => Option[Long], - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { - showMillisDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) - } - - val seconds = 1000L - val minutes = seconds * 60 - val hours = minutes * 60 + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { } - /** - * Reformat a time interval in milliseconds to a prettier format for output - */ - def millisToString(ms: Long): String = { - val (size, units) = - if (ms > hours) { - (ms.toDouble / hours, "hours") - } else if (ms > minutes) { - (ms.toDouble / minutes, "min") - } else if (ms > seconds) { - (ms.toDouble / seconds, "s") - } else { - (ms.toDouble, "ms") - } - "%.1f %s".format(size, units) - } -} + override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult): Unit = { } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { } + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit = { } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { } + + override def onBlockManagerRemoved( + blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = { } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit = { } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { } + + override def onExecutorMetricsUpdate( + executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { } + + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { } + + override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { } + + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { } -private case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) - -private object RuntimePercentage { - def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { - val denom = totalTime.toDouble - val fetchTime = metrics.shuffleReadMetrics.map(_.fetchWaitTime) - val fetch = fetchTime.map(_ / denom) - val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom - val other = 1.0 - (exec + fetch.getOrElse(0d)) - RuntimePercentage(exec, fetch, other) - } + override def onOtherEvent(event: SparkListenerEvent): Unit = { } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 94f0574f0e..471586ac08 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -22,9 +22,12 @@ import org.apache.spark.util.ListenerBus /** * A [[SparkListenerEvent]] bus that relays [[SparkListenerEvent]]s to its listeners */ -private[spark] trait SparkListenerBus extends ListenerBus[SparkListener, SparkListenerEvent] { +private[spark] trait SparkListenerBus + extends ListenerBus[SparkListenerInterface, SparkListenerEvent] { - protected override def doPostEvent(listener: SparkListener, event: SparkListenerEvent): Unit = { + protected override def doPostEvent( + listener: SparkListenerInterface, + event: SparkListenerEvent): Unit = { event match { case stageSubmitted: SparkListenerStageSubmitted => listener.onStageSubmitted(stageSubmitted) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 6e9337bb90..bc1431835e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -49,14 +49,13 @@ class SplitInfo( // So unless there is identity equality between underlyingSplits, it will always fail even if it // is pointing to same block. override def equals(other: Any): Boolean = other match { - case that: SplitInfo => { + case that: SplitInfo => this.hostLocation == that.hostLocation && this.inputFormatClazz == that.inputFormatClazz && this.path == that.path && this.length == that.length && // other split specific checks (like start for FileSplit) this.underlyingSplit == that.underlyingSplit - } case _ => false } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index a40b700cdd..b6d4e39fe5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -75,22 +75,6 @@ private[scheduler] abstract class Stage( val name: String = callSite.shortForm val details: String = callSite.longForm - private var _internalAccumulators: Seq[Accumulator[_]] = Seq.empty - - /** Internal accumulators shared across all tasks in this stage. */ - def internalAccumulators: Seq[Accumulator[_]] = _internalAccumulators - - /** - * Re-initialize the internal accumulators associated with this stage. - * - * This is called every time the stage is submitted, *except* when a subset of tasks - * belonging to this stage has already finished. Otherwise, reinitializing the internal - * accumulators here again will override partial values from the finished tasks. - */ - def resetInternalAccumulators(): Unit = { - _internalAccumulators = InternalAccumulator.create(rdd.sparkContext) - } - /** * Pointer to the [StageInfo] object for the most recent attempt. This needs to be initialized * here, before any attempts have actually been created, because the DAGScheduler uses this @@ -127,7 +111,8 @@ private[scheduler] abstract class Stage( numPartitionsToCompute: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty): Unit = { _latestInfo = StageInfo.fromStage( - this, nextAttemptId, Some(numPartitionsToCompute), taskLocalityPreferences) + this, nextAttemptId, Some(numPartitionsToCompute), + InternalAccumulator.createAll(rdd.sparkContext), taskLocalityPreferences) nextAttemptId += 1 } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 24796c1430..0fd58c41cd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashMap +import org.apache.spark.Accumulator import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.RDDInfo @@ -35,6 +36,7 @@ class StageInfo( val rddInfos: Seq[RDDInfo], val parentIds: Seq[Int], val details: String, + val internalAccumulators: Seq[Accumulator[_]] = Seq.empty, private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None @@ -42,7 +44,11 @@ class StageInfo( var completionTime: Option[Long] = None /** If the stage failed, the reason why. */ var failureReason: Option[String] = None - /** Terminal values of accumulables updated during this stage. */ + + /** + * Terminal values of accumulables updated during this stage, including all the user-defined + * accumulators. + */ val accumulables = HashMap[Long, AccumulableInfo]() def stageFailed(reason: String) { @@ -75,6 +81,7 @@ private[spark] object StageInfo { stage: Stage, attemptId: Int, numTasks: Option[Int] = None, + internalAccumulators: Seq[Accumulator[_]] = Seq.empty, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty ): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) @@ -87,6 +94,7 @@ private[spark] object StageInfo { rddInfos, stage.parents.map(_.id), stage.details, + internalAccumulators, taskLocalityPreferences) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala new file mode 100644 index 0000000000..309f4b806b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala @@ -0,0 +1,199 @@ +/* + * 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.scheduler + +import scala.collection.mutable + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging +import org.apache.spark.util.{Distribution, Utils} + + +/** + * :: DeveloperApi :: + * Simple SparkListener that logs a few summary statistics when each stage completes. + */ +@DeveloperApi +class StatsReportListener extends SparkListener with Logging { + + import org.apache.spark.scheduler.StatsReportListener._ + + private val taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val info = taskEnd.taskInfo + val metrics = taskEnd.taskMetrics + if (info != null && metrics != null) { + taskInfoMetrics += ((info, metrics)) + } + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + implicit val sc = stageCompleted + this.logInfo(s"Finished stage: ${getStatusDetail(stageCompleted.stageInfo)}") + showMillisDistribution("task runtime:", (info, _) => Some(info.duration), taskInfoMetrics) + + // Shuffle write + showBytesDistribution("shuffle bytes written:", + (_, metric) => metric.shuffleWriteMetrics.map(_.bytesWritten), taskInfoMetrics) + + // Fetch & I/O + showMillisDistribution("fetch wait time:", + (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime), taskInfoMetrics) + showBytesDistribution("remote bytes read:", + (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead), taskInfoMetrics) + showBytesDistribution("task result size:", + (_, metric) => Some(metric.resultSize), taskInfoMetrics) + + // Runtime breakdown + val runtimePcts = taskInfoMetrics.map { case (info, metrics) => + RuntimePercentage(info.duration, metrics) + } + showDistribution("executor (non-fetch) time pct: ", + Distribution(runtimePcts.map(_.executorPct * 100)), "%2.0f %%") + showDistribution("fetch wait time pct: ", + Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%") + showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%") + taskInfoMetrics.clear() + } + + private def getStatusDetail(info: StageInfo): String = { + val failureReason = info.failureReason.map("(" + _ + ")").getOrElse("") + val timeTaken = info.submissionTime.map( + x => info.completionTime.getOrElse(System.currentTimeMillis()) - x + ).getOrElse("-") + + s"Stage(${info.stageId}, ${info.attemptId}); Name: '${info.name}'; " + + s"Status: ${info.getStatusString}$failureReason; numTasks: ${info.numTasks}; " + + s"Took: $timeTaken msec" + } + +} + +private[spark] object StatsReportListener extends Logging { + + // For profiling, the extremes are more interesting + val percentiles = Array[Int](0, 5, 10, 25, 50, 75, 90, 95, 100) + val probabilities = percentiles.map(_ / 100.0) + val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" + + def extractDoubleDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Option[Double]): Option[Distribution] = { + Distribution(taskInfoMetrics.flatMap { case (info, metric) => getMetric(info, metric) }) + } + + // Is there some way to setup the types that I can get rid of this completely? + def extractLongDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Option[Long]): Option[Distribution] = { + extractDoubleDistribution( + taskInfoMetrics, + (info, metric) => { getMetric(info, metric).map(_.toDouble) }) + } + + def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { + val stats = d.statCounter + val quantiles = d.getQuantiles(probabilities).map(formatNumber) + logInfo(heading + stats) + logInfo(percentilesHeader) + logInfo("\t" + quantiles.mkString("\t")) + } + + def showDistribution( + heading: String, + dOpt: Option[Distribution], + formatNumber: Double => String) { + dOpt.foreach { d => showDistribution(heading, d, formatNumber)} + } + + def showDistribution(heading: String, dOpt: Option[Distribution], format: String) { + def f(d: Double): String = format.format(d) + showDistribution(heading, dOpt, f _) + } + + def showDistribution( + heading: String, + format: String, + getMetric: (TaskInfo, TaskMetrics) => Option[Double], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showDistribution(heading, extractDoubleDistribution(taskInfoMetrics, getMetric), format) + } + + def showBytesDistribution( + heading: String, + getMetric: (TaskInfo, TaskMetrics) => Option[Long], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showBytesDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) + } + + def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { + dOpt.foreach { dist => showBytesDistribution(heading, dist) } + } + + def showBytesDistribution(heading: String, dist: Distribution) { + showDistribution(heading, dist, (d => Utils.bytesToString(d.toLong)): Double => String) + } + + def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { + showDistribution(heading, dOpt, + (d => StatsReportListener.millisToString(d.toLong)): Double => String) + } + + def showMillisDistribution( + heading: String, + getMetric: (TaskInfo, TaskMetrics) => Option[Long], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showMillisDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) + } + + val seconds = 1000L + val minutes = seconds * 60 + val hours = minutes * 60 + + /** + * Reformat a time interval in milliseconds to a prettier format for output + */ + def millisToString(ms: Long): String = { + val (size, units) = + if (ms > hours) { + (ms.toDouble / hours, "hours") + } else if (ms > minutes) { + (ms.toDouble / minutes, "min") + } else if (ms > seconds) { + (ms.toDouble / seconds, "s") + } else { + (ms.toDouble, "ms") + } + "%.1f %s".format(size, units) + } +} + +private case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) + +private object RuntimePercentage { + def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { + val denom = totalTime.toDouble + val fetchTime = metrics.shuffleReadMetrics.map(_.fetchWaitTime) + val fetch = fetchTime.map(_ / denom) + val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom + val other = 1.0 - (exec + fetch.getOrElse(0d)) + RuntimePercentage(exec, fetch, other) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index d2b8ca90a9..1ff9d7795f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -19,12 +19,13 @@ package org.apache.spark.scheduler import java.io.{DataInputStream, DataOutputStream} import java.nio.ByteBuffer +import java.util.Properties import scala.collection.mutable.HashMap import org.apache.spark.{Accumulator, SparkEnv, TaskContext, TaskContextImpl} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.memory.TaskMemoryManager +import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} import org.apache.spark.metrics.MetricsSystem import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} @@ -46,12 +47,14 @@ import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Uti * @param initialAccumulators initial set of accumulators to be used in this task for tracking * internal metrics. Other accumulators will be registered later when * they are deserialized on the executors. + * @param localProperties copy of thread-local properties set by the user on the driver side. */ private[spark] abstract class Task[T]( val stageId: Int, val stageAttemptId: Int, val partitionId: Int, - val initialAccumulators: Seq[Accumulator[_]]) extends Serializable { + val initialAccumulators: Seq[Accumulator[_]], + @transient var localProperties: Properties) extends Serializable { /** * Called by [[org.apache.spark.executor.Executor]] to run this task. @@ -71,6 +74,7 @@ private[spark] abstract class Task[T]( taskAttemptId, attemptNumber, taskMemoryManager, + localProperties, metricsSystem, initialAccumulators) TaskContext.setTaskContext(context) @@ -80,17 +84,24 @@ private[spark] abstract class Task[T]( } try { runTask(context) - } catch { case e: Throwable => - // Catch all errors; run task failure callbacks, and rethrow the exception. - context.markTaskFailed(e) - throw e + } catch { + case e: Throwable => + // Catch all errors; run task failure callbacks, and rethrow the exception. + try { + context.markTaskFailed(e) + } catch { + case t: Throwable => + e.addSuppressed(t) + } + throw e } finally { // Call the task completion callbacks. context.markTaskCompleted() try { Utils.tryLogNonFatalError { // Release memory used by this thread for unrolling blocks - SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask() + SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP) + SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.OFF_HEAP) // Notify any tasks waiting for execution memory to be freed to wake up and try to // acquire memory again. This makes impossible the scenario where a task sleeps forever // because there are no other tasks left to notify it. Since this is safe to do but may @@ -205,6 +216,11 @@ private[spark] object Task { dataOut.writeLong(timestamp) } + // Write the task properties separately so it is available before full task deserialization. + val propBytes = Utils.serialize(task.localProperties) + dataOut.writeInt(propBytes.length) + dataOut.write(propBytes) + // Write the task itself and finish dataOut.flush() val taskBytes = serializer.serialize(task) @@ -220,7 +236,7 @@ private[spark] object Task { * @return (taskFiles, taskJars, taskBytes) */ def deserializeWithDependencies(serializedTask: ByteBuffer) - : (HashMap[String, Long], HashMap[String, Long], ByteBuffer) = { + : (HashMap[String, Long], HashMap[String, Long], Properties, ByteBuffer) = { val in = new ByteBufferInputStream(serializedTask) val dataIn = new DataInputStream(in) @@ -239,8 +255,13 @@ private[spark] object Task { taskJars(dataIn.readUTF()) = dataIn.readLong() } + val propLength = dataIn.readInt() + val propBytes = new Array[Byte](propLength) + dataIn.readFully(propBytes, 0, propLength) + val taskProps = Utils.deserialize[Properties](propBytes) + // Create a sub-buffer for the rest of the data, which is the serialized Task object val subBuffer = serializedTask.slice() // ByteBufferInputStream will have read just up to task - (taskFiles, taskJars, subBuffer) + (taskFiles, taskJars, taskProps, subBuffer) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 873f1b56bd..ae7ef46abb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -133,7 +133,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul // if we can't deserialize the reason. logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex: Exception => {} + case ex: Exception => // No-op } scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 8477a66b39..647d44a0f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -51,7 +51,7 @@ private[spark] trait TaskScheduler { def submitTasks(taskSet: TaskSet): Unit // Cancel a stage. - def cancelTasks(stageId: Int, interruptThread: Boolean) + def cancelTasks(stageId: Int, interruptThread: Boolean): Unit // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. def setDAGScheduler(dagScheduler: DAGScheduler): Unit diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index f7790fccc6..c3159188d9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -90,6 +90,8 @@ private[spark] class TaskSchedulerImpl( // Number of tasks running on each executor private val executorIdToTaskCount = new HashMap[String, Int] + def runningTasksByExecutors(): Map[String, Int] = executorIdToTaskCount.toMap + // The set of executors we have on each host; this is used to compute hostsAlive, which // in turn is used to decide when we can attain data locality on a given host protected val executorsByHost = new HashMap[String, HashSet[String]] @@ -569,6 +571,11 @@ private[spark] class TaskSchedulerImpl( return } while (!backend.isReady) { + // Might take a while for backend to be ready if it is waiting on resources. + if (sc.stopped.get) { + // For example: the master removes the application for some reason + throw new IllegalStateException("Spark context stopped while waiting for backend") + } synchronized { this.wait(100) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 15d3515a02..6e08cdd87a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -188,20 +188,18 @@ private[spark] class TaskSetManager( loc match { case e: ExecutorCacheTaskLocation => pendingTasksForExecutor.getOrElseUpdate(e.executorId, new ArrayBuffer) += index - case e: HDFSCacheTaskLocation => { + case e: HDFSCacheTaskLocation => val exe = sched.getExecutorsAliveOnHost(loc.host) exe match { - case Some(set) => { + case Some(set) => for (e <- set) { pendingTasksForExecutor.getOrElseUpdate(e, new ArrayBuffer) += index } logInfo(s"Pending task $index has a cached location at ${e.host} " + ", where there are executors " + set.mkString(",")) - } case None => logDebug(s"Pending task $index has a cached location at ${e.host} " + ", but there are no executors alive there.") } - } case _ => } pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index @@ -437,7 +435,7 @@ private[spark] class TaskSetManager( } dequeueTask(execId, host, allowedLocality) match { - case Some((index, taskLocality, speculative)) => { + case Some((index, taskLocality, speculative)) => // Found a task; do some bookkeeping and return a task description val task = tasks(index) val taskId = sched.newTaskId() @@ -486,7 +484,6 @@ private[spark] class TaskSetManager( sched.dagScheduler.taskStarted(task, info) return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, taskName, index, serializedTask)) - } case _ => } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 8d5c11dc36..46a829114e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -30,6 +30,8 @@ private[spark] object CoarseGrainedClusterMessages { case object RetrieveSparkProps extends CoarseGrainedClusterMessage + case object RetrieveLastAllocatedExecutorId extends CoarseGrainedClusterMessage + // Driver to executors case class LaunchTask(data: SerializableBuffer) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index b7919efc4b..8896391f97 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger +import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -43,24 +44,30 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp extends ExecutorAllocationClient with SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed - var totalCoreCount = new AtomicInteger(0) + protected val totalCoreCount = new AtomicInteger(0) // Total number of executors that are currently registered - var totalRegisteredExecutors = new AtomicInteger(0) - val conf = scheduler.sc.conf + protected val totalRegisteredExecutors = new AtomicInteger(0) + protected val conf = scheduler.sc.conf private val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf) // Submit tasks only after (registered resources / total expected resources) // is equal to at least this value, that is double between 0 and 1. - var minRegisteredRatio = + private val _minRegisteredRatio = math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached - val maxRegisteredWaitingTimeMs = + private val maxRegisteredWaitingTimeMs = conf.getTimeAsMs("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s") - val createTime = System.currentTimeMillis() + private val createTime = System.currentTimeMillis() + // Accessing `executorDataMap` in `DriverEndpoint.receive/receiveAndReply` doesn't need any + // protection. But accessing `executorDataMap` out of `DriverEndpoint.receive/receiveAndReply` + // must be protected by `CoarseGrainedSchedulerBackend.this`. Besides, `executorDataMap` should + // only be modified in `DriverEndpoint.receive/receiveAndReply` with protection by + // `CoarseGrainedSchedulerBackend.this`. private val executorDataMap = new HashMap[String, ExecutorData] // Number of executors requested from the cluster manager that have not registered yet + @GuardedBy("CoarseGrainedSchedulerBackend.this") private var numPendingExecutors = 0 private val listenerBus = scheduler.sc.listenerBus @@ -68,20 +75,26 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Executors we have requested the cluster manager to kill that have not died yet; maps // the executor ID to whether it was explicitly killed by the driver (and thus shouldn't // be considered an app-related failure). + @GuardedBy("CoarseGrainedSchedulerBackend.this") private val executorsPendingToRemove = new HashMap[String, Boolean] // A map to store hostname with its possible task number running on it + @GuardedBy("CoarseGrainedSchedulerBackend.this") protected var hostToLocalTaskCount: Map[String, Int] = Map.empty // The number of pending tasks which is locality required + @GuardedBy("CoarseGrainedSchedulerBackend.this") protected var localityAwareTasks = 0 - // Executors that have been lost, but for which we don't yet know the real exit reason. - protected val executorsPendingLossReason = new HashSet[String] + // The num of current max ExecutorId used to re-register appMaster + @volatile protected var currentExecutorIdCounter = 0 class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) extends ThreadSafeRpcEndpoint with Logging { + // Executors that have been lost, but for which we don't yet know the real exit reason. + protected val executorsPendingLossReason = new HashSet[String] + // If this DriverEndpoint is changed to support multiple threads, // then this may need to be changed so that we don't share the serializer // instance across threads @@ -137,7 +150,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case RegisterExecutor(executorId, executorRef, cores, logUrls) => if (executorDataMap.contains(executorId)) { - context.reply(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) + executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) + context.reply(true) } else { // If the executor's rpc env is not listening for incoming connections, `hostPort` // will be null, and the client connection should be used to contact the executor. @@ -156,13 +170,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { executorDataMap.put(executorId, data) + if (currentExecutorIdCounter < executorId.toInt) { + currentExecutorIdCounter = executorId.toInt + } if (numPendingExecutors > 0) { numPendingExecutors -= 1 logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") } } + executorRef.send(RegisteredExecutor(executorAddress.host)) // Note: some tests expect the reply to come after we put the executor in the map - context.reply(RegisteredExecutor(executorAddress.host)) + context.reply(true) listenerBus.post( SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data)) makeOffers() @@ -255,7 +273,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } // Remove a disconnected slave from the cluster - def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { + private def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { executorDataMap.get(executorId) match { case Some(executorInfo) => // This must be synchronized because variables mutated @@ -307,7 +325,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } var driverEndpoint: RpcEndpointRef = null - val taskIdsOnSlave = new HashMap[String, HashSet[String]] + + protected def minRegisteredRatio: Double = _minRegisteredRatio override def start() { val properties = new ArrayBuffer[(String, String)] @@ -356,20 +375,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Reset the state of CoarseGrainedSchedulerBackend to the initial state. Currently it will only - * be called in the yarn-client mode when AM re-registers after a failure, also dynamic - * allocation is enabled. + * be called in the yarn-client mode when AM re-registers after a failure. * */ protected def reset(): Unit = synchronized { - if (Utils.isDynamicAllocationEnabled(conf)) { - numPendingExecutors = 0 - executorsPendingToRemove.clear() - - // Remove all the lingering executors that should be removed but not yet. The reason might be - // because (1) disconnected event is not yet received; (2) executors die silently. - executorDataMap.toMap.foreach { case (eid, _) => - driverEndpoint.askWithRetry[Boolean]( - RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered."))) - } + numPendingExecutors = 0 + executorsPendingToRemove.clear() + + // Remove all the lingering executors that should be removed but not yet. The reason might be + // because (1) disconnected event is not yet received; (2) executors die silently. + executorDataMap.toMap.foreach { case (eid, _) => + driverEndpoint.askWithRetry[Boolean]( + RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered."))) } } @@ -414,7 +430,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Return the number of executors currently registered with this backend. */ - def numExistingExecutors: Int = executorDataMap.size + private def numExistingExecutors: Int = executorDataMap.size + + override def getExecutorIds(): Seq[String] = { + executorDataMap.keySet.toSeq + } /** * Request an additional number of executors from the cluster manager. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 90b1813750..50b452c72f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -295,12 +295,12 @@ private[spark] class CoarseMesosSchedulerBackend( } /** - * Launches executors on accepted offers, and declines unused offers. Executors are launched - * round-robin on offers. - * - * @param d SchedulerDriver - * @param offers Mesos offers that match attribute constraints - */ + * Launches executors on accepted offers, and declines unused offers. Executors are launched + * round-robin on offers. + * + * @param d SchedulerDriver + * @param offers Mesos offers that match attribute constraints + */ private def handleMatchedOffers(d: SchedulerDriver, offers: Buffer[Offer]): Unit = { val tasks = buildMesosTasks(offers) for (offer <- offers) { @@ -336,12 +336,12 @@ private[spark] class CoarseMesosSchedulerBackend( } /** - * Returns a map from OfferIDs to the tasks to launch on those offers. In order to maximize - * per-task memory and IO, tasks are round-robin assigned to offers. - * - * @param offers Mesos offers that match attribute constraints - * @return A map from OfferID to a list of Mesos tasks to launch on that offer - */ + * Returns a map from OfferIDs to the tasks to launch on those offers. In order to maximize + * per-task memory and IO, tasks are round-robin assigned to offers. + * + * @param offers Mesos offers that match attribute constraints + * @return A map from OfferID to a list of Mesos tasks to launch on that offer + */ private def buildMesosTasks(offers: Buffer[Offer]): Map[OfferID, List[MesosTaskInfo]] = { // offerID -> tasks val tasks = new HashMap[OfferID, List[MesosTaskInfo]].withDefaultValue(Nil) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala index 3971e6c382..61ab3e87c5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala @@ -121,11 +121,10 @@ private[spark] class ZookeeperMesosClusterPersistenceEngine( Some(Utils.deserialize[T](fileData)) } catch { case e: NoNodeException => None - case e: Exception => { + case e: Exception => logWarning("Exception while reading persisted file, deleting", e) zk.delete().forPath(zkPath) None - } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 2df7b1120b..73bd4c58e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -423,6 +423,12 @@ private[spark] class MesosClusterScheduler( "--driver-cores", desc.cores.toString, "--driver-memory", s"${desc.mem}M") + val replicatedOptionsBlacklist = Set( + "spark.jars", // Avoids duplicate classes in classpath + "spark.submit.deployMode", // this would be set to `cluster`, but we need client + "spark.master" // this contains the address of the dispatcher, not master + ) + // Assume empty main class means we're running python if (!desc.command.mainClass.equals("")) { options ++= Seq("--class", desc.command.mainClass) @@ -440,9 +446,29 @@ private[spark] class MesosClusterScheduler( .mkString(",") options ++= Seq("--py-files", formattedFiles) } + desc.schedulerProperties + .filter { case (key, _) => !replicatedOptionsBlacklist.contains(key) } + .foreach { case (key, value) => options ++= Seq("--conf", s"$key=${shellEscape(value)}") } options } + /** + * Escape args for Unix-like shells, unless already quoted by the user. + * Based on: http://www.gnu.org/software/bash/manual/html_node/Double-Quotes.html + * and http://www.grymoire.com/Unix/Quote.html + * @param value argument + * @return escaped argument + */ + private[scheduler] def shellEscape(value: String): String = { + val WrappedInQuotes = """^(".+"|'.+')$""".r + val ShellSpecialChars = (""".*([ '<>&|\?\*;!#\\(\)"$`]).*""").r + value match { + case WrappedInQuotes(c) => value // The user quoted his args, don't touch it! + case ShellSpecialChars(c) => "\"" + value.replaceAll("""(["`\$\\])""", """\\$1""") + "\"" + case _: String => value // Don't touch harmless strings + } + } + private class ResourceOffer( val offerId: OfferID, val slaveId: SlaveID, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala index 374c79a7e5..1b7ac172de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala @@ -55,11 +55,10 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging { Some(vol.setContainerPath(container_path) .setHostPath(host_path) .setMode(Volume.Mode.RO)) - case spec => { + case spec => logWarning(s"Unable to parse volume specs: $volumes. " + "Expected form: \"[host-dir:]container-dir[:rw|:ro](, ...)\"") None - } } } .map { _.build() } @@ -90,11 +89,10 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging { Some(portmap.setHostPort(host_port.toInt) .setContainerPort(container_port.toInt) .setProtocol(protocol)) - case spec => { + case spec => logWarning(s"Unable to parse port mapping specs: $portmaps. " + "Expected form: \"host_port:container_port[:udp|:tcp](, ...)\"") None - } } } .map { _.build() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 9a12a61f2f..1e322ac679 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -124,11 +124,10 @@ private[mesos] trait MesosSchedulerUtils extends Logging { markErr() } } catch { - case e: Exception => { + case e: Exception => logError("driver.run() failed", e) error = Some(e) markErr() - } } } }.start() @@ -148,8 +147,8 @@ private[mesos] trait MesosSchedulerUtils extends Logging { } /** - * Signal that the scheduler has registered with Mesos. - */ + * Signal that the scheduler has registered with Mesos. + */ protected def markRegistered(): Unit = { registerLatch.countDown() } @@ -184,7 +183,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging { var remain = amountToUse var requestedResources = new ArrayBuffer[Resource] val remainingResources = resources.asScala.map { - case r => { + case r => if (remain > 0 && r.getType == Value.Type.SCALAR && r.getScalar.getValue > 0.0 && @@ -196,7 +195,6 @@ private[mesos] trait MesosSchedulerUtils extends Logging { } else { r } - } } // Filter any resource that has depleted. @@ -228,7 +226,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging { * @return */ protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = { - offerAttributes.asScala.map(attr => { + offerAttributes.asScala.map { attr => val attrValue = attr.getType match { case Value.Type.SCALAR => attr.getScalar case Value.Type.RANGES => attr.getRanges @@ -236,7 +234,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging { case Value.Type.TEXT => attr.getText } (attr.getName, attrValue) - }).toMap + }.toMap } @@ -283,11 +281,11 @@ private[mesos] trait MesosSchedulerUtils extends Logging { * are separated by ':'. The ':' implies equality (for singular values) and "is one of" for * multiple values (comma separated). For example: * {{{ - * parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b") + * parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b") * // would result in * <code> * Map( - * "tachyon" -> Set("true"), + * "os" -> Set("centos7"), * "zone": -> Set("us-east-1a", "us-east-1b") * ) * }}} diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 3d090a4353..918ae376f6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -357,7 +357,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ * serialization. */ trait KryoRegistrator { - def registerClasses(kryo: Kryo) + def registerClasses(kryo: Kryo): Unit } private[serializer] object KryoSerializer { diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 5ead40e89e..cb95246d5b 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -188,10 +188,9 @@ abstract class DeserializationStream { try { (readKey[Any](), readValue[Any]()) } catch { - case eof: EOFException => { + case eof: EOFException => finished = true null - } } } diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 27e5fa4c2b..745ef12691 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -25,7 +25,7 @@ import scala.reflect.ClassTag import org.apache.spark.SparkConf import org.apache.spark.io.CompressionCodec import org.apache.spark.storage._ -import org.apache.spark.util.io.{ByteArrayChunkOutputStream, ChunkedByteBuffer} +import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} /** * Component which configures serialization and compression for various Spark components, including @@ -128,17 +128,9 @@ private[spark] class SerializerManager(defaultSerializer: Serializer, conf: Spar /** Serializes into a chunked byte buffer. */ def dataSerialize[T: ClassTag](blockId: BlockId, values: Iterator[T]): ChunkedByteBuffer = { - val byteArrayChunkOutputStream = new ByteArrayChunkOutputStream(1024 * 1024 * 4) - dataSerializeStream(blockId, byteArrayChunkOutputStream, values) - new ChunkedByteBuffer(byteArrayChunkOutputStream.toArrays.map(ByteBuffer.wrap)) - } - - /** - * Deserializes a ByteBuffer into an iterator of values and disposes of it when the end of - * the iterator is reached. - */ - def dataDeserialize[T: ClassTag](blockId: BlockId, bytes: ChunkedByteBuffer): Iterator[T] = { - dataDeserializeStream[T](blockId, bytes.toInputStream(dispose = true)) + val bbos = new ChunkedByteBufferOutputStream(1024 * 1024 * 4, ByteBuffer.allocate) + dataSerializeStream(blockId, bbos, values) + bbos.toChunkedByteBuffer } /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 637b2dfc19..876cdfaa87 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -69,10 +69,10 @@ private[spark] class BlockStoreShuffleReader[K, C]( // Update the context task metrics for each record read. val readMetrics = context.taskMetrics.registerTempShuffleReadMetrics() val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( - recordIter.map(record => { + recordIter.map { record => readMetrics.incRecordsRead(1) record - }), + }, context.taskMetrics().mergeShuffleReadMetrics()) // An interruptible iterator must be used here in order to support task cancellation diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index 6cd7d69518..be1e84a2ba 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -35,7 +35,7 @@ private[spark] trait ShuffleWriterGroup { val writers: Array[DiskBlockObjectWriter] /** @param success Indicates all writes were successful. If false, no blocks will be recorded. */ - def releaseWriters(success: Boolean) + def releaseWriters(success: Boolean): Unit } /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 76fd249fbd..364fad664e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -54,9 +54,9 @@ private[spark] trait ShuffleManager { context: TaskContext): ShuffleReader[K, C] /** - * Remove a shuffle's metadata from the ShuffleManager. - * @return true if the metadata removed successfully, otherwise false. - */ + * Remove a shuffle's metadata from the ShuffleManager. + * @return true if the metadata removed successfully, otherwise false. + */ def unregisterShuffle(shuffleId: Int): Boolean /** diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index 9c92a50150..f8d6e9fbbb 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -147,7 +147,7 @@ private[v1] object AllStagesResource { speculative = uiData.taskInfo.speculative, accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo }, errorMessage = uiData.errorMessage, - taskMetrics = uiData.taskMetrics.map { convertUiTaskMetrics } + taskMetrics = uiData.metrics.map { convertUiTaskMetrics } ) } @@ -155,7 +155,7 @@ private[v1] object AllStagesResource { allTaskData: Iterable[TaskUIData], quantiles: Array[Double]): TaskMetricDistributions = { - val rawMetrics = allTaskData.flatMap{_.taskMetrics}.toSeq + val rawMetrics = allTaskData.flatMap{_.metrics}.toSeq def metricQuantiles(f: InternalTaskMetrics => Double): IndexedSeq[Double] = Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 50b6ba67e9..ba9cd711f1 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -177,6 +177,12 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { @PathParam("attemptId") attemptId: String): EventLogDownloadResource = { new EventLogDownloadResource(uiRoot, appId, Some(attemptId)) } + + @Path("version") + def getVersion(): VersionResource = { + new VersionResource(uiRoot) + } + } private[spark] object ApiRootResource { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala new file mode 100644 index 0000000000..673da1ce36 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala @@ -0,0 +1,30 @@ +/* + * 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.status.api.v1 + +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class VersionResource(ui: UIRoot) { + + @GET + def getVersionInfo(): VersionInfo = new VersionInfo( + org.apache.spark.SPARK_VERSION + ) + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 909dd0c07e..ebbbf48148 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -38,7 +38,11 @@ class ApplicationAttemptInfo private[spark]( val lastUpdated: Date, val duration: Long, val sparkUser: String, - val completed: Boolean = false) + val completed: Boolean = false) { + def getStartTimeEpoch: Long = startTime.getTime + def getEndTimeEpoch: Long = endTime.getTime + def getLastUpdatedEpoch: Long = lastUpdated.getTime +} class ExecutorStageSummary private[spark]( val taskTime : Long, @@ -237,3 +241,6 @@ class AccumulableInfo private[spark]( val name: String, val update: Option[String], val value: String) + +class VersionInfo private[spark]( + val spark: String) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 0c7763f236..35a6c63ad1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.io._ +import java.nio.ByteBuffer import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, ExecutionContext, Future} @@ -39,6 +40,7 @@ import org.apache.spark.rpc.RpcEnv import org.apache.spark.serializer.{SerializerInstance, SerializerManager} import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.storage.memory._ +import org.apache.spark.unsafe.Platform import org.apache.spark.util._ import org.apache.spark.util.io.ChunkedByteBuffer @@ -372,8 +374,12 @@ private[spark] class BlockManager( val onDisk = level.useDisk && diskStore.contains(blockId) val deserialized = if (inMem) level.deserialized else false val replication = if (inMem || onDisk) level.replication else 1 - val storageLevel = - StorageLevel(onDisk, inMem, deserialized, replication) + val storageLevel = StorageLevel( + useDisk = onDisk, + useMemory = inMem, + useOffHeap = level.useOffHeap, + deserialized = deserialized, + replication = replication) val memSize = if (inMem) memoryStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L BlockStatus(storageLevel, memSize, diskSize) @@ -407,8 +413,8 @@ private[spark] class BlockManager( val iter: Iterator[Any] = if (level.deserialized) { memoryStore.getValues(blockId).get } else { - serializerManager.dataDeserialize( - blockId, memoryStore.getBytes(blockId).get)(info.classTag) + serializerManager.dataDeserializeStream( + blockId, memoryStore.getBytes(blockId).get.toInputStream())(info.classTag) } val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId)) Some(new BlockResult(ci, DataReadMethod.Memory, info.size)) @@ -416,11 +422,15 @@ private[spark] class BlockManager( val iterToReturn: Iterator[Any] = { val diskBytes = diskStore.getBytes(blockId) if (level.deserialized) { - val diskValues = serializerManager.dataDeserialize(blockId, diskBytes)(info.classTag) + val diskValues = serializerManager.dataDeserializeStream( + blockId, + diskBytes.toInputStream(dispose = true))(info.classTag) maybeCacheDiskValuesInMemory(info, blockId, level, diskValues) } else { - val bytes = maybeCacheDiskBytesInMemory(info, blockId, level, diskBytes) - serializerManager.dataDeserialize(blockId, bytes)(info.classTag) + val stream = maybeCacheDiskBytesInMemory(info, blockId, level, diskBytes) + .map {_.toInputStream(dispose = false)} + .getOrElse { diskBytes.toInputStream(dispose = true) } + serializerManager.dataDeserializeStream(blockId, stream)(info.classTag) } } val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, releaseLock(blockId)) @@ -481,7 +491,8 @@ private[spark] class BlockManager( if (level.useMemory && memoryStore.contains(blockId)) { memoryStore.getBytes(blockId).get } else if (level.useDisk && diskStore.contains(blockId)) { - maybeCacheDiskBytesInMemory(info, blockId, level, diskStore.getBytes(blockId)) + val diskBytes = diskStore.getBytes(blockId) + maybeCacheDiskBytesInMemory(info, blockId, level, diskBytes).getOrElse(diskBytes) } else { releaseLock(blockId) throw new SparkException(s"Block $blockId was not found even though it's read-locked") @@ -496,8 +507,9 @@ private[spark] class BlockManager( */ private def getRemoteValues(blockId: BlockId): Option[BlockResult] = { getRemoteBytes(blockId).map { data => - new BlockResult( - serializerManager.dataDeserialize(blockId, data), DataReadMethod.Network, data.size) + val values = + serializerManager.dataDeserializeStream(blockId, data.toInputStream(dispose = true)) + new BlockResult(values, DataReadMethod.Network, data.size) } } @@ -631,6 +643,14 @@ private[spark] class BlockManager( level: StorageLevel, classTag: ClassTag[T], makeIterator: () => Iterator[T]): Either[BlockResult, Iterator[T]] = { + // Attempt to read the block from local or remote storage. If it's present, then we don't need + // to go through the local-get-or-put path. + get(blockId) match { + case Some(block) => + return Left(block) + case _ => + // Need to compute the block. + } // Initially we hold no locks on this block. doPutIterator(blockId, makeIterator, level, classTag, keepReadLock = true) match { case None => @@ -745,7 +765,8 @@ private[spark] class BlockManager( // Put it in memory first, even if it also has useDisk set to true; // We will drop it to disk later if the memory store can't hold it. val putSucceeded = if (level.deserialized) { - val values = serializerManager.dataDeserialize(blockId, bytes)(classTag) + val values = + serializerManager.dataDeserializeStream(blockId, bytes.toInputStream())(classTag) memoryStore.putIteratorAsValues(blockId, values, classTag) match { case Right(_) => true case Left(iter) => @@ -755,7 +776,7 @@ private[spark] class BlockManager( false } } else { - memoryStore.putBytes(blockId, size, () => bytes) + memoryStore.putBytes(blockId, size, level.memoryMode, () => bytes) } if (!putSucceeded && level.useDisk) { logWarning(s"Persisting block $blockId to disk instead.") @@ -893,7 +914,7 @@ private[spark] class BlockManager( } } } else { // !level.deserialized - memoryStore.putIteratorAsBytes(blockId, iterator(), classTag) match { + memoryStore.putIteratorAsBytes(blockId, iterator(), classTag, level.memoryMode) match { case Right(s) => size = s case Left(partiallySerializedValues) => @@ -951,14 +972,16 @@ private[spark] class BlockManager( * Attempts to cache spilled bytes read from disk into the MemoryStore in order to speed up * subsequent reads. This method requires the caller to hold a read lock on the block. * - * @return a copy of the bytes. The original bytes passed this method should no longer - * be used after this method returns. + * @return a copy of the bytes from the memory store if the put succeeded, otherwise None. + * If this returns bytes from the memory store then the original disk store bytes will + * automatically be disposed and the caller should not continue to use them. Otherwise, + * if this returns None then the original disk store bytes will be unaffected. */ private def maybeCacheDiskBytesInMemory( blockInfo: BlockInfo, blockId: BlockId, level: StorageLevel, - diskBytes: ChunkedByteBuffer): ChunkedByteBuffer = { + diskBytes: ChunkedByteBuffer): Option[ChunkedByteBuffer] = { require(!level.deserialized) if (level.useMemory) { // Synchronize on blockInfo to guard against a race condition where two readers both try to @@ -966,25 +989,29 @@ private[spark] class BlockManager( blockInfo.synchronized { if (memoryStore.contains(blockId)) { diskBytes.dispose() - memoryStore.getBytes(blockId).get + Some(memoryStore.getBytes(blockId).get) } else { - val putSucceeded = memoryStore.putBytes(blockId, diskBytes.size, () => { + val allocator = level.memoryMode match { + case MemoryMode.ON_HEAP => ByteBuffer.allocate _ + case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _ + } + val putSucceeded = memoryStore.putBytes(blockId, diskBytes.size, level.memoryMode, () => { // https://issues.apache.org/jira/browse/SPARK-6076 // If the file size is bigger than the free memory, OOM will happen. So if we // cannot put it into MemoryStore, copyForMemory should not be created. That's why // this action is put into a `() => ChunkedByteBuffer` and created lazily. - diskBytes.copy() + diskBytes.copy(allocator) }) if (putSucceeded) { diskBytes.dispose() - memoryStore.getBytes(blockId).get + Some(memoryStore.getBytes(blockId).get) } else { - diskBytes + None } } } } else { - diskBytes + None } } @@ -1055,7 +1082,12 @@ private[spark] class BlockManager( val peersForReplication = new ArrayBuffer[BlockManagerId] val peersReplicatedTo = new ArrayBuffer[BlockManagerId] val peersFailedToReplicateTo = new ArrayBuffer[BlockManagerId] - val tLevel = StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1) + val tLevel = StorageLevel( + useDisk = level.useDisk, + useMemory = level.useMemory, + useOffHeap = level.useOffHeap, + deserialized = level.deserialized, + replication = 1) val startTime = System.currentTimeMillis val random = new Random(blockId.hashCode) @@ -1264,9 +1296,12 @@ private[spark] class BlockManager( "the disk, memory, or external block store") } blockInfoManager.removeBlock(blockId) + val removeBlockStatus = getCurrentBlockStatus(blockId, info) if (tellMaster && info.tellMaster) { - val status = getCurrentBlockStatus(blockId, info) - reportBlockStatus(blockId, info, status) + reportBlockStatus(blockId, info, removeBlockStatus) + } + Option(TaskContext.get()).foreach { c => + c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, removeBlockStatus))) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index d2a5c69e15..8fa1215011 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -453,7 +453,7 @@ private[spark] class BlockManagerInfo( } if (storageLevel.isValid) { - /* isValid means it is either stored in-memory, on-disk or on-externalBlockStore. + /* isValid means it is either stored in-memory or on-disk. * The memSize here indicates the data size in or dropped from memory, * externalBlockStoreSize here indicates the data size in or dropped from externalBlockStore, * and the diskSize here indicates the data size in or dropped to disk. diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 25edb9f1e4..4ec5b4bbb0 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -143,13 +143,12 @@ final class ShuffleBlockFetcherIterator( while (iter.hasNext) { val result = iter.next() result match { - case SuccessFetchResult(_, address, _, buf, _) => { + case SuccessFetchResult(_, address, _, buf, _) => if (address != blockManager.blockManagerId) { shuffleMetrics.incRemoteBytesRead(buf.size) shuffleMetrics.incRemoteBlocksFetched(1) } buf.release() - } case _ => } } @@ -313,7 +312,7 @@ final class ShuffleBlockFetcherIterator( shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait) result match { - case SuccessFetchResult(_, address, size, buf, isNetworkReqDone) => { + case SuccessFetchResult(_, address, size, buf, isNetworkReqDone) => if (address != blockManager.blockManagerId) { shuffleMetrics.incRemoteBytesRead(buf.size) shuffleMetrics.incRemoteBlocksFetched(1) @@ -323,7 +322,6 @@ final class ShuffleBlockFetcherIterator( reqsInFlight -= 1 logDebug("Number of requests in flight " + reqsInFlight) } - } case _ => } // Send fetch requests up to maxBytesInFlight diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 7d23295e25..216ec07934 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -60,10 +60,7 @@ class StorageLevel private( assert(replication < 40, "Replication restricted to be less than 40 for calculating hash codes") if (useOffHeap) { - require(!useDisk, "Off-heap storage level does not support using disk") - require(!useMemory, "Off-heap storage level does not support using heap memory") require(!deserialized, "Off-heap storage level does not support deserialized storage") - require(replication == 1, "Off-heap storage level does not support multiple replication") } private[spark] def memoryMode: MemoryMode = { @@ -86,7 +83,7 @@ class StorageLevel private( false } - def isValid: Boolean = (useMemory || useDisk || useOffHeap) && (replication > 0) + def isValid: Boolean = (useMemory || useDisk) && (replication > 0) def toInt: Int = { var ret = 0 @@ -123,7 +120,8 @@ class StorageLevel private( private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) override def toString: String = { - s"StorageLevel($useDisk, $useMemory, $useOffHeap, $deserialized, $replication)" + s"StorageLevel(disk=$useDisk, memory=$useMemory, offheap=$useOffHeap, " + + s"deserialized=$deserialized, replication=$replication)" } override def hashCode(): Int = toInt * 41 + replication @@ -131,8 +129,9 @@ class StorageLevel private( def description: String = { var result = "" result += (if (useDisk) "Disk " else "") - result += (if (useMemory) "Memory " else "") - result += (if (useOffHeap) "ExternalBlockStore " else "") + if (useMemory) { + result += (if (useOffHeap) "Memory (off heap) " else "Memory ") + } result += (if (deserialized) "Deserialized " else "Serialized ") result += s"${replication}x Replicated" result @@ -156,9 +155,7 @@ object StorageLevel { val MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2) val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false) val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) - - // Redirect to MEMORY_ONLY_SER for now. - val OFF_HEAP = MEMORY_ONLY_SER + val OFF_HEAP = new StorageLevel(true, true, true, false, 1) /** * :: DeveloperApi :: @@ -183,7 +180,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object without setting useOffHeap. + * Create a new StorageLevel object. */ @DeveloperApi def apply( @@ -198,7 +195,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object. + * Create a new StorageLevel object without setting useOffHeap. */ @DeveloperApi def apply( diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index f552b498a7..3008520f61 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -66,17 +66,6 @@ class StorageStatusListener(conf: SparkConf) extends SparkListener { } } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { - val info = taskEnd.taskInfo - val metrics = taskEnd.taskMetrics - if (info != null && metrics != null) { - val updatedBlocks = metrics.updatedBlockStatuses - if (updatedBlocks.length > 0) { - updateStorageStatus(info.executorId, updatedBlocks) - } - } - } - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { updateStorageStatus(unpersistRDD.rddId) } @@ -102,4 +91,14 @@ class StorageStatusListener(conf: SparkConf) extends SparkListener { } } } + + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { + val executorId = blockUpdated.blockUpdatedInfo.blockManagerId.executorId + val blockId = blockUpdated.blockUpdatedInfo.blockId + val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel + val memSize = blockUpdated.blockUpdatedInfo.memSize + val diskSize = blockUpdated.blockUpdatedInfo.diskSize + val blockStatus = BlockStatus(storageLevel, memSize, diskSize) + updateStorageStatus(executorId, Seq((blockId, blockStatus))) + } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 199a5fc270..fb9941bbd9 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -175,7 +175,10 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { def memRemaining: Long = maxMem - memUsed /** Return the memory used by this block manager. */ - def memUsed: Long = _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).sum + def memUsed: Long = _nonRddStorageInfo._1 + cacheSize + + /** Return the memory used by caching RDDs */ + def cacheSize: Long = _rddBlocks.keys.toSeq.map(memUsedByRdd).sum /** Return the disk space used by this block manager. */ def diskUsed: Long = _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 3ca41f32c1..99be4de065 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -32,20 +32,25 @@ import org.apache.spark.internal.Logging import org.apache.spark.memory.{MemoryManager, MemoryMode} import org.apache.spark.serializer.{SerializationStream, SerializerManager} import org.apache.spark.storage.{BlockId, BlockInfoManager, StorageLevel} +import org.apache.spark.unsafe.Platform import org.apache.spark.util.{CompletionIterator, SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector -import org.apache.spark.util.io.{ByteArrayChunkOutputStream, ChunkedByteBuffer} +import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} private sealed trait MemoryEntry[T] { def size: Long + def memoryMode: MemoryMode def classTag: ClassTag[T] } private case class DeserializedMemoryEntry[T]( value: Array[T], size: Long, - classTag: ClassTag[T]) extends MemoryEntry[T] + classTag: ClassTag[T]) extends MemoryEntry[T] { + val memoryMode: MemoryMode = MemoryMode.ON_HEAP +} private case class SerializedMemoryEntry[T]( buffer: ChunkedByteBuffer, + memoryMode: MemoryMode, classTag: ClassTag[T]) extends MemoryEntry[T] { def size: Long = buffer.size } @@ -86,7 +91,10 @@ private[spark] class MemoryStore( // A mapping from taskAttemptId to amount of memory used for unrolling a block (in bytes) // All accesses of this map are assumed to have manually synchronized on `memoryManager` - private val unrollMemoryMap = mutable.HashMap[Long, Long]() + private val onHeapUnrollMemoryMap = mutable.HashMap[Long, Long]() + // Note: off-heap unroll memory is only used in putIteratorAsBytes() because off-heap caching + // always stores serialized values. + private val offHeapUnrollMemoryMap = mutable.HashMap[Long, Long]() // Initial memory to request before unrolling any block private val unrollMemoryThreshold: Long = @@ -131,13 +139,14 @@ private[spark] class MemoryStore( def putBytes[T: ClassTag]( blockId: BlockId, size: Long, + memoryMode: MemoryMode, _bytes: () => ChunkedByteBuffer): Boolean = { require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") - if (memoryManager.acquireStorageMemory(blockId, size, MemoryMode.ON_HEAP)) { + if (memoryManager.acquireStorageMemory(blockId, size, memoryMode)) { // We acquired enough memory for the block, so go ahead and put it val bytes = _bytes() assert(bytes.size == size) - val entry = new SerializedMemoryEntry[T](bytes, implicitly[ClassTag[T]]) + val entry = new SerializedMemoryEntry[T](bytes, memoryMode, implicitly[ClassTag[T]]) entries.synchronized { entries.put(blockId, entry) } @@ -190,7 +199,8 @@ private[spark] class MemoryStore( var vector = new SizeTrackingVector[T]()(classTag) // Request enough memory to begin unrolling - keepUnrolling = reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold) + keepUnrolling = + reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, MemoryMode.ON_HEAP) if (!keepUnrolling) { logWarning(s"Failed to reserve initial memory threshold of " + @@ -207,7 +217,8 @@ private[spark] class MemoryStore( val currentSize = vector.estimateSize() if (currentSize >= memoryThreshold) { val amountToRequest = (currentSize * memoryGrowthFactor - memoryThreshold).toLong - keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest) + keepUnrolling = + reserveUnrollMemoryForThisTask(blockId, amountToRequest, MemoryMode.ON_HEAP) if (keepUnrolling) { unrollMemoryUsedByThisBlock += amountToRequest } @@ -228,7 +239,7 @@ private[spark] class MemoryStore( def transferUnrollToStorage(amount: Long): Unit = { // Synchronize so that transfer is atomic memoryManager.synchronized { - releaseUnrollMemoryForThisTask(amount) + releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, amount) val success = memoryManager.acquireStorageMemory(blockId, amount, MemoryMode.ON_HEAP) assert(success, "transferring unroll memory to storage memory failed") } @@ -247,7 +258,7 @@ private[spark] class MemoryStore( // If this task attempt already owns more unroll memory than is necessary to store the // block, then release the extra memory that will not be used. val excessUnrollMemory = unrollMemoryUsedByThisBlock - size - releaseUnrollMemoryForThisTask(excessUnrollMemory) + releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, excessUnrollMemory) transferUnrollToStorage(size) true } @@ -295,10 +306,16 @@ private[spark] class MemoryStore( private[storage] def putIteratorAsBytes[T]( blockId: BlockId, values: Iterator[T], - classTag: ClassTag[T]): Either[PartiallySerializedBlock[T], Long] = { + classTag: ClassTag[T], + memoryMode: MemoryMode): Either[PartiallySerializedBlock[T], Long] = { require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") + val allocator = memoryMode match { + case MemoryMode.ON_HEAP => ByteBuffer.allocate _ + case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _ + } + // Whether there is still enough memory for us to continue unrolling this block var keepUnrolling = true // Initial per-task memory to request for unrolling blocks (bytes). @@ -307,15 +324,15 @@ private[spark] class MemoryStore( var unrollMemoryUsedByThisBlock = 0L // Underlying buffer for unrolling the block val redirectableStream = new RedirectableOutputStream - val byteArrayChunkOutputStream = new ByteArrayChunkOutputStream(initialMemoryThreshold.toInt) - redirectableStream.setOutputStream(byteArrayChunkOutputStream) + val bbos = new ChunkedByteBufferOutputStream(initialMemoryThreshold.toInt, allocator) + redirectableStream.setOutputStream(bbos) val serializationStream: SerializationStream = { val ser = serializerManager.getSerializer(classTag).newInstance() ser.serializeStream(serializerManager.wrapForCompression(blockId, redirectableStream)) } // Request enough memory to begin unrolling - keepUnrolling = reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold) + keepUnrolling = reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, memoryMode) if (!keepUnrolling) { logWarning(s"Failed to reserve initial memory threshold of " + @@ -325,9 +342,9 @@ private[spark] class MemoryStore( } def reserveAdditionalMemoryIfNecessary(): Unit = { - if (byteArrayChunkOutputStream.size > unrollMemoryUsedByThisBlock) { - val amountToRequest = byteArrayChunkOutputStream.size - unrollMemoryUsedByThisBlock - keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest) + if (bbos.size > unrollMemoryUsedByThisBlock) { + val amountToRequest = bbos.size - unrollMemoryUsedByThisBlock + keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest, memoryMode) if (keepUnrolling) { unrollMemoryUsedByThisBlock += amountToRequest } @@ -349,12 +366,11 @@ private[spark] class MemoryStore( } if (keepUnrolling) { - val entry = SerializedMemoryEntry[T]( - new ChunkedByteBuffer(byteArrayChunkOutputStream.toArrays.map(ByteBuffer.wrap)), classTag) + val entry = SerializedMemoryEntry[T](bbos.toChunkedByteBuffer, memoryMode, classTag) // Synchronize so that transfer is atomic memoryManager.synchronized { - releaseUnrollMemoryForThisTask(unrollMemoryUsedByThisBlock) - val success = memoryManager.acquireStorageMemory(blockId, entry.size, MemoryMode.ON_HEAP) + releaseUnrollMemoryForThisTask(memoryMode, unrollMemoryUsedByThisBlock) + val success = memoryManager.acquireStorageMemory(blockId, entry.size, memoryMode) assert(success, "transferring unroll memory to storage memory failed") } entries.synchronized { @@ -365,7 +381,7 @@ private[spark] class MemoryStore( Right(entry.size) } else { // We ran out of space while unrolling the values for this block - logUnrollFailureMessage(blockId, byteArrayChunkOutputStream.size) + logUnrollFailureMessage(blockId, bbos.size) Left( new PartiallySerializedBlock( this, @@ -374,7 +390,8 @@ private[spark] class MemoryStore( serializationStream, redirectableStream, unrollMemoryUsedByThisBlock, - new ChunkedByteBuffer(byteArrayChunkOutputStream.toArrays.map(ByteBuffer.wrap)), + memoryMode, + bbos.toChunkedByteBuffer, values, classTag)) } @@ -386,7 +403,7 @@ private[spark] class MemoryStore( case null => None case e: DeserializedMemoryEntry[_] => throw new IllegalArgumentException("should only call getBytes on serialized blocks") - case SerializedMemoryEntry(bytes, _) => Some(bytes) + case SerializedMemoryEntry(bytes, _, _) => Some(bytes) } } @@ -407,8 +424,12 @@ private[spark] class MemoryStore( entries.remove(blockId) } if (entry != null) { - memoryManager.releaseStorageMemory(entry.size, MemoryMode.ON_HEAP) - logInfo(s"Block $blockId of size ${entry.size} dropped " + + entry match { + case SerializedMemoryEntry(buffer, _, _) => buffer.dispose() + case _ => + } + memoryManager.releaseStorageMemory(entry.size, entry.memoryMode) + logDebug(s"Block $blockId of size ${entry.size} dropped " + s"from memory (free ${maxMemory - blocksMemoryUsed})") true } else { @@ -420,7 +441,8 @@ private[spark] class MemoryStore( entries.synchronized { entries.clear() } - unrollMemoryMap.clear() + onHeapUnrollMemoryMap.clear() + offHeapUnrollMemoryMap.clear() memoryManager.releaseAllStorageMemory() logInfo("MemoryStore cleared") } @@ -433,23 +455,27 @@ private[spark] class MemoryStore( } /** - * Try to evict blocks to free up a given amount of space to store a particular block. - * Can fail if either the block is bigger than our memory or it would require replacing - * another block from the same RDD (which leads to a wasteful cyclic replacement pattern for - * RDDs that don't fit into memory that we want to avoid). - * - * @param blockId the ID of the block we are freeing space for, if any - * @param space the size of this block - * @return the amount of memory (in bytes) freed by eviction - */ - private[spark] def evictBlocksToFreeSpace(blockId: Option[BlockId], space: Long): Long = { + * Try to evict blocks to free up a given amount of space to store a particular block. + * Can fail if either the block is bigger than our memory or it would require replacing + * another block from the same RDD (which leads to a wasteful cyclic replacement pattern for + * RDDs that don't fit into memory that we want to avoid). + * + * @param blockId the ID of the block we are freeing space for, if any + * @param space the size of this block + * @param memoryMode the type of memory to free (on- or off-heap) + * @return the amount of memory (in bytes) freed by eviction + */ + private[spark] def evictBlocksToFreeSpace( + blockId: Option[BlockId], + space: Long, + memoryMode: MemoryMode): Long = { assert(space > 0) memoryManager.synchronized { var freedMemory = 0L val rddToAdd = blockId.flatMap(getRddId) val selectedBlocks = new ArrayBuffer[BlockId] - def blockIsEvictable(blockId: BlockId): Boolean = { - rddToAdd.isEmpty || rddToAdd != getRddId(blockId) + def blockIsEvictable(blockId: BlockId, entry: MemoryEntry[_]): Boolean = { + entry.memoryMode == memoryMode && (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) } // This is synchronized to ensure that the set of entries is not changed // (because of getValue or getBytes) while traversing the iterator, as that @@ -459,7 +485,8 @@ private[spark] class MemoryStore( while (freedMemory < space && iterator.hasNext) { val pair = iterator.next() val blockId = pair.getKey - if (blockIsEvictable(blockId)) { + val entry = pair.getValue + if (blockIsEvictable(blockId, entry)) { // We don't want to evict blocks which are currently being read, so we need to obtain // an exclusive write lock on blocks which are candidates for eviction. We perform a // non-blocking "tryLock" here in order to ignore blocks which are locked for reading: @@ -474,7 +501,7 @@ private[spark] class MemoryStore( def dropBlock[T](blockId: BlockId, entry: MemoryEntry[T]): Unit = { val data = entry match { case DeserializedMemoryEntry(values, _, _) => Left(values) - case SerializedMemoryEntry(buffer, _) => Right(buffer) + case SerializedMemoryEntry(buffer, _, _) => Right(buffer) } val newEffectiveStorageLevel = blockEvictionHandler.dropFromMemory(blockId, () => data)(entry.classTag) @@ -530,11 +557,18 @@ private[spark] class MemoryStore( * * @return whether the request is granted. */ - def reserveUnrollMemoryForThisTask(blockId: BlockId, memory: Long): Boolean = { + def reserveUnrollMemoryForThisTask( + blockId: BlockId, + memory: Long, + memoryMode: MemoryMode): Boolean = { memoryManager.synchronized { - val success = memoryManager.acquireUnrollMemory(blockId, memory, MemoryMode.ON_HEAP) + val success = memoryManager.acquireUnrollMemory(blockId, memory, memoryMode) if (success) { val taskAttemptId = currentTaskAttemptId() + val unrollMemoryMap = memoryMode match { + case MemoryMode.ON_HEAP => onHeapUnrollMemoryMap + case MemoryMode.OFF_HEAP => offHeapUnrollMemoryMap + } unrollMemoryMap(taskAttemptId) = unrollMemoryMap.getOrElse(taskAttemptId, 0L) + memory } success @@ -545,9 +579,13 @@ private[spark] class MemoryStore( * Release memory used by this task for unrolling blocks. * If the amount is not specified, remove the current task's allocation altogether. */ - def releaseUnrollMemoryForThisTask(memory: Long = Long.MaxValue): Unit = { + def releaseUnrollMemoryForThisTask(memoryMode: MemoryMode, memory: Long = Long.MaxValue): Unit = { val taskAttemptId = currentTaskAttemptId() memoryManager.synchronized { + val unrollMemoryMap = memoryMode match { + case MemoryMode.ON_HEAP => onHeapUnrollMemoryMap + case MemoryMode.OFF_HEAP => offHeapUnrollMemoryMap + } if (unrollMemoryMap.contains(taskAttemptId)) { val memoryToRelease = math.min(memory, unrollMemoryMap(taskAttemptId)) if (memoryToRelease > 0) { @@ -555,7 +593,7 @@ private[spark] class MemoryStore( if (unrollMemoryMap(taskAttemptId) == 0) { unrollMemoryMap.remove(taskAttemptId) } - memoryManager.releaseUnrollMemory(memoryToRelease, MemoryMode.ON_HEAP) + memoryManager.releaseUnrollMemory(memoryToRelease, memoryMode) } } } @@ -565,20 +603,23 @@ private[spark] class MemoryStore( * Return the amount of memory currently occupied for unrolling blocks across all tasks. */ def currentUnrollMemory: Long = memoryManager.synchronized { - unrollMemoryMap.values.sum + onHeapUnrollMemoryMap.values.sum + offHeapUnrollMemoryMap.values.sum } /** * Return the amount of memory currently occupied for unrolling blocks by this task. */ def currentUnrollMemoryForThisTask: Long = memoryManager.synchronized { - unrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L) + onHeapUnrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L) + + offHeapUnrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L) } /** * Return the number of tasks currently unrolling blocks. */ - private def numTasksUnrolling: Int = memoryManager.synchronized { unrollMemoryMap.keys.size } + private def numTasksUnrolling: Int = memoryManager.synchronized { + (onHeapUnrollMemoryMap.keys ++ offHeapUnrollMemoryMap.keys).toSet.size + } /** * Log information about current memory usage. @@ -627,7 +668,7 @@ private[storage] class PartiallyUnrolledIterator[T]( private[this] var iter: Iterator[T] = { val completionIterator = CompletionIterator[T, Iterator[T]](unrolled, { unrolledIteratorIsConsumed = true - memoryStore.releaseUnrollMemoryForThisTask(unrollMemory) + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) }) completionIterator ++ rest } @@ -640,7 +681,7 @@ private[storage] class PartiallyUnrolledIterator[T]( */ def close(): Unit = { if (!unrolledIteratorIsConsumed) { - memoryStore.releaseUnrollMemoryForThisTask(unrollMemory) + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) unrolledIteratorIsConsumed = true } iter = null @@ -669,6 +710,7 @@ private class RedirectableOutputStream extends OutputStream { * @param serializationStream a serialization stream which writes to [[redirectableOutputStream]]. * @param redirectableOutputStream an OutputStream which can be redirected to a different sink. * @param unrollMemory the amount of unroll memory used by the values in `unrolled`. + * @param memoryMode whether the unroll memory is on- or off-heap * @param unrolled a byte buffer containing the partially-serialized values. * @param rest the rest of the original iterator passed to * [[MemoryStore.putIteratorAsValues()]]. @@ -681,18 +723,36 @@ private[storage] class PartiallySerializedBlock[T]( serializationStream: SerializationStream, redirectableOutputStream: RedirectableOutputStream, unrollMemory: Long, + memoryMode: MemoryMode, unrolled: ChunkedByteBuffer, rest: Iterator[T], classTag: ClassTag[T]) { + // If the task does not fully consume `valuesIterator` or otherwise fails to consume or dispose of + // this PartiallySerializedBlock then we risk leaking of direct buffers, so we use a task + // completion listener here in order to ensure that `unrolled.dispose()` is called at least once. + // The dispose() method is idempotent, so it's safe to call it unconditionally. + Option(TaskContext.get()).foreach { taskContext => + taskContext.addTaskCompletionListener { _ => + // When a task completes, its unroll memory will automatically be freed. Thus we do not call + // releaseUnrollMemoryForThisTask() here because we want to avoid double-freeing. + unrolled.dispose() + } + } + /** * Called to dispose of this block and free its memory. */ def discard(): Unit = { try { + // We want to close the output stream in order to free any resources associated with the + // serializer itself (such as Kryo's internal buffers). close() might cause data to be + // written, so redirect the output stream to discard that data. + redirectableOutputStream.setOutputStream(ByteStreams.nullOutputStream()) serializationStream.close() } finally { - memoryStore.releaseUnrollMemoryForThisTask(unrollMemory) + unrolled.dispose() + memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) } } @@ -701,12 +761,14 @@ private[storage] class PartiallySerializedBlock[T]( * and then serializing the values from the original input iterator. */ def finishWritingToStream(os: OutputStream): Unit = { - ByteStreams.copy(unrolled.toInputStream(), os) + // `unrolled`'s underlying buffers will be freed once this input stream is fully read: + ByteStreams.copy(unrolled.toInputStream(dispose = true), os) + memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) redirectableOutputStream.setOutputStream(os) while (rest.hasNext) { serializationStream.writeObject(rest.next())(classTag) } - discard() + serializationStream.close() } /** @@ -717,10 +779,13 @@ private[storage] class PartiallySerializedBlock[T]( * `close()` on it to free its resources. */ def valuesIterator: PartiallyUnrolledIterator[T] = { + // `unrolled`'s underlying buffers will be freed once this input stream is fully read: + val unrolledIter = serializerManager.dataDeserializeStream( + blockId, unrolled.toInputStream(dispose = true))(classTag) new PartiallyUnrolledIterator( memoryStore, unrollMemory, - unrolled = serializerManager.dataDeserialize(blockId, unrolled)(classTag), + unrolled = CompletionIterator[T, Iterator[T]](unrolledIter, discard()), rest = rest) } } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index c3c59f857d..119165f724 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -30,6 +30,7 @@ import org.eclipse.jetty.server.handler._ import org.eclipse.jetty.server.nio.SelectChannelConnector import org.eclipse.jetty.server.ssl.SslSelectChannelConnector import org.eclipse.jetty.servlet._ +import org.eclipse.jetty.util.component.LifeCycle import org.eclipse.jetty.util.thread.QueuedThreadPool import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} @@ -350,4 +351,15 @@ private[spark] object JettyUtils extends Logging { private[spark] case class ServerInfo( server: Server, boundPort: Int, - rootHandler: ContextHandlerCollection) + rootHandler: ContextHandlerCollection) { + + def stop(): Unit = { + server.stop() + // Stop the ThreadPool if it supports stop() method (through LifeCycle). + // It is needed because stopping the Server won't stop the ThreadPool it uses. + val threadPool = server.getThreadPool + if (threadPool != null && threadPool.isInstanceOf[LifeCycle]) { + threadPool.asInstanceOf[LifeCycle].stop + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 6057522509..39155ff264 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -80,6 +80,10 @@ private[spark] class SparkUI private ( } initialize() + def getSparkUser: String = { + environmentListener.systemProperties.toMap.get("user.name").getOrElse("<unknown>") + } + def getAppName: String = appName def setAppId(id: String): Unit = { diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 250b7f2e5f..2b0bc32cf6 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -129,7 +129,7 @@ private[spark] abstract class WebUI( } /** Initialize all components of the server. */ - def initialize() + def initialize(): Unit /** Bind to the HTTP server behind this web interface. */ def bind() { @@ -153,7 +153,7 @@ private[spark] abstract class WebUI( def stop() { assert(serverInfo.isDefined, "Attempted to stop %s before binding to a server!".format(className)) - serverInfo.get.server.stop() + serverInfo.get.stop() } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala index cc476d61b5..a0ef80d9bd 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -38,7 +38,7 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage val content = maybeThreadDump.map { threadDump => val dumpRows = threadDump.sortWith { - case (threadTrace1, threadTrace2) => { + case (threadTrace1, threadTrace2) => val v1 = if (threadTrace1.threadName.contains("Executor task launch")) 1 else 0 val v2 = if (threadTrace2.threadName.contains("Executor task launch")) 1 else 0 if (v1 == v2) { @@ -46,7 +46,6 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage } else { v1 > v2 } - } }.map { thread => val threadId = thread.threadId <tr id={s"thread_${threadId}_tr"} class="accordion-heading" diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 788f35ec77..3fd0efd3a1 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -70,7 +70,7 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap executorToTotalCores(eid) = executorAdded.executorInfo.totalCores executorToTasksMax(eid) = executorToTotalCores(eid) / conf.getInt("spark.task.cpus", 1) - executorIdToData(eid) = ExecutorUIData(executorAdded.time) + executorIdToData(eid) = new ExecutorUIData(executorAdded.time) } override def onExecutorRemoved( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index d1c8b3089a..07484c9550 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -148,7 +148,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { | 'Removed at ${UIUtils.formatDate(new Date(event.finishTime.get))}' + | '${ if (event.finishReason.isDefined) { - s"""<br>Reason: ${event.finishReason.get}""" + s"""<br>Reason: ${event.finishReason.get.replace("\n", " ")}""" } else { "" } @@ -297,6 +297,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { <div> <ul class="unstyled"> <li> + <strong>User:</strong> + {parent.getSparkUser} + </li> + <li> <strong>Total Uptime:</strong> { if (endTime < 0 && parent.sc.isDefined) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 1304efd8f2..f609fb4cd2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -42,13 +42,13 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage var hasShuffleWrite = false var hasShuffleRead = false var hasBytesSpilled = false - stageData.foreach(data => { + stageData.foreach { data => hasInput = data.hasInput hasOutput = data.hasOutput hasShuffleRead = data.hasShuffleRead hasShuffleWrite = data.hasShuffleWrite hasBytesSpilled = data.hasBytesSpilled - }) + } <table class={UIUtils.TABLE_CLASS_STRIPED_SORTABLE}> <thead> diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 654d988807..bd4797ae8e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -122,7 +122,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { | 'Removed at ${UIUtils.formatDate(new Date(event.finishTime.get))}' + | '${ if (event.finishReason.isDefined) { - s"""<br>Reason: ${event.finishReason.get}""" + s"""<br>Reason: ${event.finishReason.get.replace("\n", " ")}""" } else { "" } @@ -203,7 +203,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { // This could be empty if the JobProgressListener hasn't received information about the // stage or if the stage information has been garbage collected listener.stageIdToInfo.getOrElse(stageId, - new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown")) + new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown", Seq.empty)) } val activeStages = Buffer[StageInfo]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ed3ab66e3b..13f5f84d06 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -396,13 +396,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { None } taskMetrics.foreach { m => - val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.taskMetrics) + val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics) updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) } val taskData = stageData.taskData.getOrElseUpdate(info.taskId, new TaskUIData(info)) taskData.taskInfo = info - taskData.taskMetrics = taskMetrics + taskData.metrics = taskMetrics taskData.errorMessage = errorMessage for ( @@ -506,9 +506,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val metrics = TaskMetrics.fromAccumulatorUpdates(accumUpdates) taskData.foreach { t => if (!t.taskInfo.finished) { - updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.taskMetrics) + updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.metrics) // Overwrite task metrics - t.taskMetrics = Some(metrics) + t.metrics = Some(metrics) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index 0d0e9b00d3..7b00b558d5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -31,6 +31,8 @@ private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { def isFairScheduler: Boolean = jobProgresslistener.schedulingMode == Some(SchedulingMode.FAIR) + def getSparkUser: String = parent.getSparkUser + attachPage(new AllJobsPage(this)) attachPage(new JobPage(this)) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 689ab7dd5e..8a44bbd9fc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -330,7 +330,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { else taskTable.dataSource.slicedTaskIds // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) + val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.metrics.isDefined) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { @@ -348,8 +348,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { getDistributionQuantiles(data).map(d => <td>{Utils.bytesToString(d.toLong)}</td>) } - val deserializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.executorDeserializeTime.toDouble + val deserializationTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.executorDeserializeTime.toDouble } val deserializationQuantiles = <td> @@ -359,13 +359,13 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { </span> </td> +: getFormattedTimeQuantiles(deserializationTimes) - val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.executorRunTime.toDouble + val serviceTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.executorRunTime.toDouble } val serviceQuantiles = <td>Duration</td> +: getFormattedTimeQuantiles(serviceTimes) - val gcTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.jvmGCTime.toDouble + val gcTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.jvmGCTime.toDouble } val gcQuantiles = <td> @@ -374,8 +374,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { </span> </td> +: getFormattedTimeQuantiles(gcTimes) - val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.resultSerializationTime.toDouble + val serializationTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.resultSerializationTime.toDouble } val serializationQuantiles = <td> @@ -385,8 +385,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { </span> </td> +: getFormattedTimeQuantiles(serializationTimes) - val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => - getGettingResultTime(info, currentTime).toDouble + val gettingResultTimes = validTasks.map { taskUIData: TaskUIData => + getGettingResultTime(taskUIData.taskInfo, currentTime).toDouble } val gettingResultQuantiles = <td> @@ -397,8 +397,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { </td> +: getFormattedTimeQuantiles(gettingResultTimes) - val peakExecutionMemory = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.peakExecutionMemory.toDouble + val peakExecutionMemory = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.peakExecutionMemory.toDouble } val peakExecutionMemoryQuantiles = { <td> @@ -412,8 +412,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). - val schedulerDelays = validTasks.map { case TaskUIData(info, metrics, _) => - getSchedulerDelay(info, metrics.get, currentTime).toDouble + val schedulerDelays = validTasks.map { taskUIData: TaskUIData => + getSchedulerDelay(taskUIData.taskInfo, taskUIData.metrics.get, currentTime).toDouble } val schedulerDelayTitle = <td><span data-toggle="tooltip" title={ToolTips.SCHEDULER_DELAY} data-placement="right">Scheduler Delay</span></td> @@ -427,30 +427,30 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ) } - val inputSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble + val inputSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble } - val inputRecords = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.inputMetrics.map(_.recordsRead).getOrElse(0L).toDouble + val inputRecords = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.inputMetrics.map(_.recordsRead).getOrElse(0L).toDouble } val inputQuantiles = <td>Input Size / Records</td> +: getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords) - val outputSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.outputMetrics.map(_.bytesWritten).getOrElse(0L).toDouble + val outputSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.outputMetrics.map(_.bytesWritten).getOrElse(0L).toDouble } - val outputRecords = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.outputMetrics.map(_.recordsWritten).getOrElse(0L).toDouble + val outputRecords = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.outputMetrics.map(_.recordsWritten).getOrElse(0L).toDouble } val outputQuantiles = <td>Output Size / Records</td> +: getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords) - val shuffleReadBlockedTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleReadMetrics.map(_.fetchWaitTime).getOrElse(0L).toDouble + val shuffleReadBlockedTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleReadMetrics.map(_.fetchWaitTime).getOrElse(0L).toDouble } val shuffleReadBlockedQuantiles = <td> @@ -461,11 +461,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { </td> +: getFormattedTimeQuantiles(shuffleReadBlockedTimes) - val shuffleReadTotalSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L).toDouble + val shuffleReadTotalSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L).toDouble } - val shuffleReadTotalRecords = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L).toDouble + val shuffleReadTotalRecords = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L).toDouble } val shuffleReadTotalQuantiles = <td> @@ -476,8 +476,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { </td> +: getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords) - val shuffleReadRemoteSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + val shuffleReadRemoteSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } val shuffleReadRemoteQuantiles = <td> @@ -488,25 +488,25 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { </td> +: getFormattedSizeQuantiles(shuffleReadRemoteSizes) - val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleWriteMetrics.map(_.bytesWritten).getOrElse(0L).toDouble + val shuffleWriteSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleWriteMetrics.map(_.bytesWritten).getOrElse(0L).toDouble } - val shuffleWriteRecords = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleWriteMetrics.map(_.recordsWritten).getOrElse(0L).toDouble + val shuffleWriteRecords = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleWriteMetrics.map(_.recordsWritten).getOrElse(0L).toDouble } val shuffleWriteQuantiles = <td>Shuffle Write Size / Records</td> +: getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords) - val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.memoryBytesSpilled.toDouble + val memoryBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.memoryBytesSpilled.toDouble } val memoryBytesSpilledQuantiles = <td>Shuffle spill (memory)</td> +: getFormattedSizeQuantiles(memoryBytesSpilledSizes) - val diskBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.diskBytesSpilled.toDouble + val diskBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.diskBytesSpilled.toDouble } val diskBytesSpilledQuantiles = <td>Shuffle spill (disk)</td> +: getFormattedSizeQuantiles(diskBytesSpilledSizes) @@ -601,7 +601,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { def toProportion(time: Long) = time.toDouble / totalExecutionTime * 100 - val metricsOpt = taskUIData.taskMetrics + val metricsOpt = taskUIData.metrics val shuffleReadTime = metricsOpt.flatMap(_.shuffleReadMetrics.map(_.fetchWaitTime)).getOrElse(0L) val shuffleReadTimeProportion = toProportion(shuffleReadTime) @@ -868,7 +868,8 @@ private[ui] class TaskDataSource( def slicedTaskIds: Set[Long] = _slicedTaskIds private def taskRow(taskData: TaskUIData): TaskTableRowData = { - val TaskUIData(info, metrics, errorMessage) = taskData + val info = taskData.taskInfo + val metrics = taskData.metrics val duration = if (info.status == "RUNNING") info.timeRunning(currentTime) else metrics.map(_.executorRunTime).getOrElse(1L) val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) @@ -1014,7 +1015,7 @@ private[ui] class TaskDataSource( shuffleRead, shuffleWrite, bytesSpilled, - errorMessage.getOrElse("")) + taskData.errorMessage.getOrElse("")) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 78165d7b74..b454ef1b20 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -105,12 +105,12 @@ private[spark] object UIData { /** * These are kept mutable and reused throughout a task's lifetime to avoid excessive reallocation. */ - case class TaskUIData( + class TaskUIData( var taskInfo: TaskInfo, - var taskMetrics: Option[TaskMetrics] = None, + var metrics: Option[TaskMetrics] = None, var errorMessage: Option[String] = None) - case class ExecutorUIData( + class ExecutorUIData( val startTime: Long, var finishTime: Option[Long] = None, var finishReason: Option[String] = None) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 8f75b586e1..50095831b4 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -57,17 +57,6 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc StorageUtils.updateRddInfo(rddInfosToUpdate, activeStorageStatusList) } - /** - * Assumes the storage status list is fully up-to-date. This implies the corresponding - * StorageStatusSparkListener must process the SparkListenerTaskEnd event before this listener. - */ - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { - val metrics = taskEnd.taskMetrics - if (metrics != null && metrics.updatedBlockStatuses.nonEmpty) { - updateRDDInfo(metrics.updatedBlockStatuses) - } - } - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { val rddInfos = stageSubmitted.stageInfo.rddInfos rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } @@ -84,4 +73,14 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { _rddInfoMap.remove(unpersistRDD.rddId) } + + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { + super.onBlockUpdated(blockUpdated) + val blockId = blockUpdated.blockUpdatedInfo.blockId + val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel + val memSize = blockUpdated.blockUpdatedInfo.memSize + val diskSize = blockUpdated.blockUpdatedInfo.diskSize + val blockStatus = BlockStatus(storageLevel, memSize, diskSize) + updateRDDInfo(Seq((blockId, blockStatus))) + } } diff --git a/core/src/main/scala/org/apache/spark/util/CausedBy.scala b/core/src/main/scala/org/apache/spark/util/CausedBy.scala new file mode 100644 index 0000000000..73df446d98 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/CausedBy.scala @@ -0,0 +1,36 @@ +/* + * 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 + +/** + * Extractor Object for pulling out the root cause of an error. + * If the error contains no cause, it will return the error itself. + * + * Usage: + * try { + * ... + * } catch { + * case CausedBy(ex: CommitDeniedException) => ... + * } + */ +private[spark] object CausedBy { + + def unapply(e: Throwable): Option[Throwable] = { + Option(e.getCause).flatMap(cause => unapply(cause)).orElse(Some(e)) + } +} 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 f4772a9803..489688cb08 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -19,7 +19,8 @@ package org.apache.spark.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} -import scala.collection.mutable.{Map, Set} +import scala.collection.mutable.{Map, Set, Stack} +import scala.language.existentials import org.apache.xbean.asm5.{ClassReader, ClassVisitor, MethodVisitor, Type} import org.apache.xbean.asm5.Opcodes._ @@ -77,15 +78,14 @@ private[spark] object ClosureCleaner extends Logging { */ private def getInnerClosureClasses(obj: AnyRef): List[Class[_]] = { val seen = Set[Class[_]](obj.getClass) - var stack = List[Class[_]](obj.getClass) + val stack = Stack[Class[_]](obj.getClass) while (!stack.isEmpty) { - val cr = getClassReader(stack.head) - stack = stack.tail + val cr = getClassReader(stack.pop()) val set = Set[Class[_]]() cr.accept(new InnerClosureFinder(set), 0) for (cls <- set -- seen) { seen += cls - stack = cls :: stack + stack.push(cls) } } (seen - obj.getClass).toList @@ -218,16 +218,24 @@ private[spark] object ClosureCleaner extends Logging { // Note that all outer objects but the outermost one (first one in this list) must be closures var outerPairs: List[(Class[_], AnyRef)] = (outerClasses zip outerObjects).reverse var parent: AnyRef = null - if (outerPairs.size > 0 && !isClosure(outerPairs.head._1)) { - // The closure is ultimately nested inside a class; keep the object of that - // class without cloning it since we don't want to clone the user's objects. - // Note that we still need to keep around the outermost object itself because - // we need it to clone its child closure later (see below). - logDebug(s" + outermost object is not a closure, so do not clone it: ${outerPairs.head}") - parent = outerPairs.head._2 // e.g. SparkContext - outerPairs = outerPairs.tail - } else if (outerPairs.size > 0) { - logDebug(s" + outermost object is a closure, so we just keep it: ${outerPairs.head}") + if (outerPairs.size > 0) { + val (outermostClass, outermostObject) = outerPairs.head + if (isClosure(outermostClass)) { + logDebug(s" + outermost object is a closure, so we clone it: ${outerPairs.head}") + } else if (outermostClass.getName.startsWith("$line")) { + // SPARK-14558: if the outermost object is a REPL line object, we should clone and clean it + // as it may carray a lot of unnecessary information, e.g. hadoop conf, spark conf, etc. + logDebug(s" + outermost object is a REPL line object, so we clone it: ${outerPairs.head}") + } else { + // The closure is ultimately nested inside a class; keep the object of that + // class without cloning it since we don't want to clone the user's objects. + // Note that we still need to keep around the outermost object itself because + // we need it to clone its child closure later (see below). + logDebug(" + outermost object is not a closure or REPL line object, so do not clone it: " + + outerPairs.head) + parent = outermostObject // e.g. SparkContext + outerPairs = outerPairs.tail + } } else { logDebug(" + there are no enclosing objects!") } diff --git a/core/src/main/scala/org/apache/spark/util/EventLoop.scala b/core/src/main/scala/org/apache/spark/util/EventLoop.scala index 153025cef2..3ea9139e11 100644 --- a/core/src/main/scala/org/apache/spark/util/EventLoop.scala +++ b/core/src/main/scala/org/apache/spark/util/EventLoop.scala @@ -47,13 +47,12 @@ private[spark] abstract class EventLoop[E](name: String) extends Logging { try { onReceive(event) } catch { - case NonFatal(e) => { + case NonFatal(e) => try { onError(e) } catch { case NonFatal(e) => logError("Unexpected error in " + name, e) } - } } } } catch { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 09d955300a..558767e36f 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -578,7 +578,9 @@ private[spark] object JsonProtocol { // The "Stage Infos" field was added in Spark 1.2.0 val stageInfos = Utils.jsonOption(json \ "Stage Infos") .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { - stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown")) + stageIds.map { id => + new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", Seq.empty) + } } SparkListenerJobStart(jobId, submissionTime, stageInfos, properties) } @@ -686,7 +688,7 @@ private[spark] object JsonProtocol { } val stageInfo = new StageInfo( - stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details) + stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details, Seq.empty) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason @@ -811,8 +813,8 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Input Metrics").foreach { inJson => val readMethod = DataReadMethod.withName((inJson \ "Data Read Method").extract[String]) val inputMetrics = metrics.registerInputMetrics(readMethod) - inputMetrics.incBytesReadInternal((inJson \ "Bytes Read").extract[Long]) - inputMetrics.incRecordsReadInternal((inJson \ "Records Read").extractOpt[Long].getOrElse(0L)) + inputMetrics.incBytesRead((inJson \ "Bytes Read").extract[Long]) + inputMetrics.incRecordsRead((inJson \ "Records Read").extractOpt[Long].getOrElse(0L)) } // Updated blocks 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 3f627a0145..6861a75612 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -151,13 +151,12 @@ object SizeEstimator extends Logging { // TODO: We could use reflection on the VMOption returned ? getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true") } catch { - case e: Exception => { + case e: Exception => // Guess whether they've enabled UseCompressedOops based on whether maxMemory < 32 GB val guess = Runtime.getRuntime.maxMemory < (32L*1024*1024*1024) val guessInWords = if (guess) "yes" else "not" logWarning("Failed to check whether UseCompressedOops is set; assuming " + guessInWords) return guess - } } } diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala new file mode 100644 index 0000000000..4dcf95177a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala @@ -0,0 +1,112 @@ +/* + * 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 javax.annotation.concurrent.GuardedBy + +/** + * A special Thread that provides "runUninterruptibly" to allow running codes without being + * interrupted by `Thread.interrupt()`. If `Thread.interrupt()` is called during runUninterruptibly + * is running, it won't set the interrupted status. Instead, setting the interrupted status will be + * deferred until it's returning from "runUninterruptibly". + * + * Note: "runUninterruptibly" should be called only in `this` thread. + */ +private[spark] class UninterruptibleThread(name: String) extends Thread(name) { + + /** A monitor to protect "uninterruptible" and "interrupted" */ + private val uninterruptibleLock = new Object + + /** + * Indicates if `this` thread are in the uninterruptible status. If so, interrupting + * "this" will be deferred until `this` enters into the interruptible status. + */ + @GuardedBy("uninterruptibleLock") + private var uninterruptible = false + + /** + * Indicates if we should interrupt `this` when we are leaving the uninterruptible zone. + */ + @GuardedBy("uninterruptibleLock") + private var shouldInterruptThread = false + + /** + * Run `f` uninterruptibly in `this` thread. The thread won't be interrupted before returning + * from `f`. + * + * If this method finds that `interrupt` is called before calling `f` and it's not inside another + * `runUninterruptibly`, it will throw `InterruptedException`. + * + * Note: this method should be called only in `this` thread. + */ + def runUninterruptibly[T](f: => T): T = { + if (Thread.currentThread() != this) { + throw new IllegalStateException(s"Call runUninterruptibly in a wrong thread. " + + s"Expected: $this but was ${Thread.currentThread()}") + } + + if (uninterruptibleLock.synchronized { uninterruptible }) { + // We are already in the uninterruptible status. So just run "f" and return + return f + } + + uninterruptibleLock.synchronized { + // Clear the interrupted status if it's set. + if (Thread.interrupted() || shouldInterruptThread) { + shouldInterruptThread = false + // Since it's interrupted, we don't need to run `f` which may be a long computation. + // Throw InterruptedException as we don't have a T to return. + throw new InterruptedException() + } + uninterruptible = true + } + try { + f + } finally { + uninterruptibleLock.synchronized { + uninterruptible = false + if (shouldInterruptThread) { + // Recover the interrupted status + super.interrupt() + shouldInterruptThread = false + } + } + } + } + + /** + * Tests whether `interrupt()` has been called. + */ + override def isInterrupted: Boolean = { + super.isInterrupted || uninterruptibleLock.synchronized { shouldInterruptThread } + } + + /** + * Interrupt `this` thread if possible. If `this` is in the uninterruptible status, it won't be + * interrupted until it enters into the interruptible status. + */ + override def interrupt(): Unit = { + uninterruptibleLock.synchronized { + if (uninterruptible) { + shouldInterruptThread = true + } else { + super.interrupt() + } + } + } +} 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 73768ff4c8..78e164cff7 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -256,10 +256,11 @@ private[spark] object Utils extends Logging { dir } - /** Copy all data from an InputStream to an OutputStream. NIO way of file stream to file stream - * copying is disabled by default unless explicitly set transferToEnabled as true, - * the parameter transferToEnabled should be configured by spark.file.transferTo = [true|false]. - */ + /** + * Copy all data from an InputStream to an OutputStream. NIO way of file stream to file stream + * copying is disabled by default unless explicitly set transferToEnabled as true, + * the parameter transferToEnabled should be configured by spark.file.transferTo = [true|false]. + */ def copyStream(in: InputStream, out: OutputStream, closeStreams: Boolean = false, @@ -1120,9 +1121,9 @@ private[spark] object Utils extends Logging { extraEnvironment: Map[String, String] = Map.empty, redirectStderr: Boolean = true): String = { val process = executeCommand(command, workingDir, extraEnvironment, redirectStderr) - val output = new StringBuffer + val output = new StringBuilder val threadName = "read stdout for " + command(0) - def appendToOutput(s: String): Unit = output.append(s) + def appendToOutput(s: String): Unit = output.append(s).append("\n") val stdoutThread = processStreamByLine(threadName, process.getInputStream, appendToOutput) val exitCode = process.waitFor() stdoutThread.join() // Wait for it to finish reading output @@ -1259,26 +1260,35 @@ private[spark] object Utils extends Logging { } /** - * Execute a block of code, call the failure callbacks before finally block if there is any - * exceptions happen. But if exceptions happen in the finally block, do not suppress the original - * exception. + * Execute a block of code and call the failure callbacks in the catch block. If exceptions occur + * in either the catch or the finally block, they are appended to the list of suppressed + * exceptions in original exception which is then rethrown. * - * This is primarily an issue with `finally { out.close() }` blocks, where - * close needs to be called to clean up `out`, but if an exception happened - * in `out.write`, it's likely `out` may be corrupted and `out.close` will + * This is primarily an issue with `catch { abort() }` or `finally { out.close() }` blocks, + * where the abort/close needs to be called to clean up `out`, but if an exception happened + * in `out.write`, it's likely `out` may be corrupted and `abort` or `out.close` will * fail as well. This would then suppress the original/likely more meaningful * exception from the original `out.write` call. */ - def tryWithSafeFinallyAndFailureCallbacks[T](block: => T)(finallyBlock: => Unit): T = { + def tryWithSafeFinallyAndFailureCallbacks[T](block: => T) + (catchBlock: => Unit = (), finallyBlock: => Unit = ()): T = { var originalThrowable: Throwable = null try { block } catch { - case t: Throwable => + case cause: Throwable => // Purposefully not using NonFatal, because even fatal exceptions // we don't want to have our finallyBlock suppress - originalThrowable = t - TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(t) + originalThrowable = cause + try { + logError("Aborting task", originalThrowable) + TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(originalThrowable) + catchBlock + } catch { + case t: Throwable => + originalThrowable.addSuppressed(t) + logWarning(s"Suppressing exception in catch: " + t.getMessage, t) + } throw originalThrowable } finally { try { @@ -1564,9 +1574,11 @@ private[spark] object Utils extends Logging { else -1 } - /** Returns the system properties map that is thread-safe to iterator over. It gets the - * properties which have been set explicitly, as well as those for which only a default value - * has been defined. */ + /** + * Returns the system properties map that is thread-safe to iterator over. It gets the + * properties which have been set explicitly, as well as those for which only a default value + * has been defined. + */ def getSystemProperties: Map[String, String] = { System.getProperties.stringPropertyNames().asScala .map(key => (key, System.getProperty(key))).toMap 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 c643c4b63c..fb4706e78d 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 @@ -41,6 +41,8 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { require(chunks.forall(_.limit() > 0), "chunks must be non-empty") require(chunks.forall(_.position() == 0), "chunks' positions must be 0") + private[this] var disposed: Boolean = false + /** * This size of this buffer, in bytes. */ @@ -117,11 +119,12 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { /** * Make a copy of this ChunkedByteBuffer, copying all of the backing data into new buffers. * The new buffer will share no resources with the original buffer. + * + * @param allocator a method for allocating byte buffers */ - def copy(): ChunkedByteBuffer = { + def copy(allocator: Int => ByteBuffer): ChunkedByteBuffer = { val copiedChunks = getChunks().map { chunk => - // TODO: accept an allocator in this copy method to integrate with mem. accounting systems - val newChunk = ByteBuffer.allocate(chunk.limit()) + val newChunk = allocator(chunk.limit()) newChunk.put(chunk) newChunk.flip() newChunk @@ -136,7 +139,10 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { * unfortunately no standard API to do this. */ def dispose(): Unit = { - chunks.foreach(StorageUtils.dispose) + if (!disposed) { + chunks.foreach(StorageUtils.dispose) + disposed = true + } } } diff --git a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala index 16fe3be303..67b50d1e70 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala @@ -18,19 +18,25 @@ package org.apache.spark.util.io import java.io.OutputStream +import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer +import org.apache.spark.storage.StorageUtils /** * An OutputStream that writes to fixed-size chunks of byte arrays. * * @param chunkSize size of each chunk, in bytes. */ -private[spark] -class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { +private[spark] class ChunkedByteBufferOutputStream( + chunkSize: Int, + allocator: Int => ByteBuffer) + extends OutputStream { - private[this] val chunks = new ArrayBuffer[Array[Byte]] + private[this] var toChunkedByteBufferWasCalled = false + + private val chunks = new ArrayBuffer[ByteBuffer] /** Index of the last chunk. Starting with -1 when the chunks array is empty. */ private[this] var lastChunkIndex = -1 @@ -48,7 +54,7 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { override def write(b: Int): Unit = { allocateNewChunkIfNeeded() - chunks(lastChunkIndex)(position) = b.toByte + chunks(lastChunkIndex).put(b.toByte) position += 1 _size += 1 } @@ -58,7 +64,7 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { while (written < len) { allocateNewChunkIfNeeded() val thisBatch = math.min(chunkSize - position, len - written) - System.arraycopy(bytes, written + off, chunks(lastChunkIndex), position, thisBatch) + chunks(lastChunkIndex).put(bytes, written + off, thisBatch) written += thisBatch position += thisBatch } @@ -67,33 +73,41 @@ class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { @inline private def allocateNewChunkIfNeeded(): Unit = { + require(!toChunkedByteBufferWasCalled, "cannot write after toChunkedByteBuffer() is called") if (position == chunkSize) { - chunks += new Array[Byte](chunkSize) + chunks += allocator(chunkSize) lastChunkIndex += 1 position = 0 } } - def toArrays: Array[Array[Byte]] = { + def toChunkedByteBuffer: ChunkedByteBuffer = { + require(!toChunkedByteBufferWasCalled, "toChunkedByteBuffer() can only be called once") + toChunkedByteBufferWasCalled = true if (lastChunkIndex == -1) { - new Array[Array[Byte]](0) + new ChunkedByteBuffer(Array.empty[ByteBuffer]) } else { // Copy the first n-1 chunks to the output, and then create an array that fits the last chunk. // An alternative would have been returning an array of ByteBuffers, with the last buffer // bounded to only the last chunk's position. However, given our use case in Spark (to put // the chunks in block manager), only limiting the view bound of the buffer would still // require the block manager to store the whole chunk. - val ret = new Array[Array[Byte]](chunks.size) + val ret = new Array[ByteBuffer](chunks.size) for (i <- 0 until chunks.size - 1) { ret(i) = chunks(i) + ret(i).flip() } if (position == chunkSize) { ret(lastChunkIndex) = chunks(lastChunkIndex) + ret(lastChunkIndex).flip() } else { - ret(lastChunkIndex) = new Array[Byte](position) - System.arraycopy(chunks(lastChunkIndex), 0, ret(lastChunkIndex), 0, position) + ret(lastChunkIndex) = allocator(position) + chunks(lastChunkIndex).flip() + ret(lastChunkIndex).put(chunks(lastChunkIndex)) + ret(lastChunkIndex).flip() + StorageUtils.dispose(chunks(lastChunkIndex)) } - ret + new ChunkedByteBuffer(ret) } } } diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala index b34880d3a7..6e80db2f51 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -32,10 +32,10 @@ private[spark] trait RollingPolicy { def shouldRollover(bytesToBeWritten: Long): Boolean /** Notify that rollover has occurred */ - def rolledOver() + def rolledOver(): Unit /** Notify that bytes have been written */ - def bytesWritten(bytes: Long) + def bytesWritten(bytes: Long): Unit /** Get the desired name of the rollover file */ def generateRolledOverFileSuffix(): String diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index 70f3dd62b9..41f28f6e51 100644 --- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala @@ -26,5 +26,5 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi trait Pseudorandom { /** Set random seed. */ - def setSeed(seed: Long) + def setSeed(seed: Long): Unit } diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 3c61528ab5..8c67364ef1 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -39,7 +39,14 @@ import org.apache.spark.annotation.DeveloperApi trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ - def sample(items: Iterator[T]): Iterator[U] + def sample(items: Iterator[T]): Iterator[U] = + items.filter(_ => sample > 0).asInstanceOf[Iterator[U]] + + /** + * Whether to sample the next item or not. + * Return how many times the next item will be sampled. Return 0 if it is not sampled. + */ + def sample(): Int /** return a copy of the RandomSampler object */ override def clone: RandomSampler[T, U] = @@ -107,21 +114,13 @@ class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = fals override def setSeed(seed: Long): Unit = rng.setSeed(seed) - override def sample(items: Iterator[T]): Iterator[T] = { + override def sample(): Int = { if (ub - lb <= 0.0) { - if (complement) items else Iterator.empty + if (complement) 1 else 0 } else { - if (complement) { - items.filter { item => { - val x = rng.nextDouble() - (x < lb) || (x >= ub) - }} - } else { - items.filter { item => { - val x = rng.nextDouble() - (x >= lb) && (x < ub) - }} - } + val x = rng.nextDouble() + val n = if ((x >= lb) && (x < ub)) 1 else 0 + if (complement) 1 - n else n } } @@ -155,15 +154,22 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T override def setSeed(seed: Long): Unit = rng.setSeed(seed) - override def sample(items: Iterator[T]): Iterator[T] = { + private lazy val gapSampling: GapSampling = + new GapSampling(fraction, rng, RandomSampler.rngEpsilon) + + override def sample(): Int = { if (fraction <= 0.0) { - Iterator.empty + 0 } else if (fraction >= 1.0) { - items + 1 } else if (fraction <= RandomSampler.defaultMaxGapSamplingFraction) { - new GapSamplingIterator(items, fraction, rng, RandomSampler.rngEpsilon) + gapSampling.sample() } else { - items.filter { _ => rng.nextDouble() <= fraction } + if (rng.nextDouble() <= fraction) { + 1 + } else { + 0 + } } } @@ -180,7 +186,7 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T * @tparam T item type */ @DeveloperApi -class PoissonSampler[T: ClassTag]( +class PoissonSampler[T]( fraction: Double, useGapSamplingIfPossible: Boolean) extends RandomSampler[T, T] { @@ -201,15 +207,29 @@ class PoissonSampler[T: ClassTag]( rngGap.setSeed(seed) } - override def sample(items: Iterator[T]): Iterator[T] = { + private lazy val gapSamplingReplacement = + new GapSamplingReplacement(fraction, rngGap, RandomSampler.rngEpsilon) + + override def sample(): Int = { if (fraction <= 0.0) { - Iterator.empty + 0 } else if (useGapSamplingIfPossible && fraction <= RandomSampler.defaultMaxGapSamplingFraction) { - new GapSamplingReplacementIterator(items, fraction, rngGap, RandomSampler.rngEpsilon) + gapSamplingReplacement.sample() } else { + rng.sample() + } + } + + override def sample(items: Iterator[T]): Iterator[T] = { + if (fraction <= 0.0) { + Iterator.empty + } else { + val useGapSampling = useGapSamplingIfPossible && + fraction <= RandomSampler.defaultMaxGapSamplingFraction + items.flatMap { item => - val count = rng.sample() + val count = if (useGapSampling) gapSamplingReplacement.sample() else rng.sample() if (count == 0) Iterator.empty else Iterator.fill(count)(item) } } @@ -220,50 +240,36 @@ class PoissonSampler[T: ClassTag]( private[spark] -class GapSamplingIterator[T: ClassTag]( - var data: Iterator[T], +class GapSampling( f: Double, rng: Random = RandomSampler.newDefaultRNG, - epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] { + epsilon: Double = RandomSampler.rngEpsilon) extends Serializable { require(f > 0.0 && f < 1.0, s"Sampling fraction ($f) must reside on open interval (0, 1)") require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0") - /** implement efficient linear-sequence drop until Scala includes fix for jira SI-8835. */ - private val iterDrop: Int => Unit = { - val arrayClass = Array.empty[T].iterator.getClass - val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass - data.getClass match { - case `arrayClass` => - (n: Int) => { data = data.drop(n) } - case `arrayBufferClass` => - (n: Int) => { data = data.drop(n) } - case _ => - (n: Int) => { - var j = 0 - while (j < n && data.hasNext) { - data.next() - j += 1 - } - } - } - } - - override def hasNext: Boolean = data.hasNext + private val lnq = math.log1p(-f) - override def next(): T = { - val r = data.next() - advance() - r + /** Return 1 if the next item should be sampled. Otherwise, return 0. */ + def sample(): Int = { + if (countForDropping > 0) { + countForDropping -= 1 + 0 + } else { + advance() + 1 + } } - private val lnq = math.log1p(-f) + private var countForDropping: Int = 0 - /** skip elements that won't be sampled, according to geometric dist P(k) = (f)(1-f)^k. */ + /** + * Decide the number of elements that won't be sampled, + * according to geometric dist P(k) = (f)(1-f)^k. + */ private def advance(): Unit = { val u = math.max(rng.nextDouble(), epsilon) - val k = (math.log(u) / lnq).toInt - iterDrop(k) + countForDropping = (math.log(u) / lnq).toInt } /** advance to first sample as part of object construction. */ @@ -273,73 +279,24 @@ class GapSamplingIterator[T: ClassTag]( // work reliably. } + private[spark] -class GapSamplingReplacementIterator[T: ClassTag]( - var data: Iterator[T], - f: Double, - rng: Random = RandomSampler.newDefaultRNG, - epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] { +class GapSamplingReplacement( + val f: Double, + val rng: Random = RandomSampler.newDefaultRNG, + epsilon: Double = RandomSampler.rngEpsilon) extends Serializable { require(f > 0.0, s"Sampling fraction ($f) must be > 0") require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0") - /** implement efficient linear-sequence drop until scala includes fix for jira SI-8835. */ - private val iterDrop: Int => Unit = { - val arrayClass = Array.empty[T].iterator.getClass - val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass - data.getClass match { - case `arrayClass` => - (n: Int) => { data = data.drop(n) } - case `arrayBufferClass` => - (n: Int) => { data = data.drop(n) } - case _ => - (n: Int) => { - var j = 0 - while (j < n && data.hasNext) { - data.next() - j += 1 - } - } - } - } - - /** current sampling value, and its replication factor, as we are sampling with replacement. */ - private var v: T = _ - private var rep: Int = 0 - - override def hasNext: Boolean = data.hasNext || rep > 0 - - override def next(): T = { - val r = v - rep -= 1 - if (rep <= 0) advance() - r - } - - /** - * Skip elements with replication factor zero (i.e. elements that won't be sampled). - * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is - * q is the probability of Poisson(0; f) - */ - private def advance(): Unit = { - val u = math.max(rng.nextDouble(), epsilon) - val k = (math.log(u) / (-f)).toInt - iterDrop(k) - // set the value and replication factor for the next value - if (data.hasNext) { - v = data.next() - rep = poissonGE1 - } - } - - private val q = math.exp(-f) + protected val q = math.exp(-f) /** * Sample from Poisson distribution, conditioned such that the sampled value is >= 1. * This is an adaptation from the algorithm for Generating Poisson distributed random variables: * http://en.wikipedia.org/wiki/Poisson_distribution */ - private def poissonGE1: Int = { + protected def poissonGE1: Int = { // simulate that the standard poisson sampling // gave us at least one iteration, for a sample of >= 1 var pp = q + ((1.0 - q) * rng.nextDouble()) @@ -353,6 +310,28 @@ class GapSamplingReplacementIterator[T: ClassTag]( } r } + private var countForDropping: Int = 0 + + def sample(): Int = { + if (countForDropping > 0) { + countForDropping -= 1 + 0 + } else { + val r = poissonGE1 + advance() + r + } + } + + /** + * Skip elements with replication factor zero (i.e. elements that won't be sampled). + * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is + * q is the probability of Poisson(0; f) + */ + private def advance(): Unit = { + val u = math.max(rng.nextDouble(), epsilon) + countForDropping = (math.log(u) / (-f)).toInt + } /** advance to first sample as part of object construction. */ advance() 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 44733dcdaf..30750b1bf1 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 @@ -170,11 +170,11 @@ public class UnsafeShuffleWriterSuite { private UnsafeShuffleWriter<Object, Object> createWriter( boolean transferToEnabled) throws IOException { conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); - return new UnsafeShuffleWriter<Object, Object>( + return new UnsafeShuffleWriter<>( blockManager, shuffleBlockResolver, taskMemoryManager, - new SerializedShuffleHandle<Object, Object>(0, 1, shuffleDep), + new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, conf 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 449fb45c30..84b82f5a47 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 @@ -182,7 +182,7 @@ public abstract class AbstractBytesToBytesMapSuite { public void emptyMap() { BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, 64, PAGE_SIZE_BYTES); try { - Assert.assertEquals(0, map.numElements()); + Assert.assertEquals(0, map.numKeys()); final int keyLengthInWords = 10; final int keyLengthInBytes = keyLengthInWords * 8; final byte[] key = getRandomByteArray(keyLengthInWords); @@ -204,7 +204,7 @@ public abstract class AbstractBytesToBytesMapSuite { final BytesToBytesMap.Location loc = map.lookup(keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes); Assert.assertFalse(loc.isDefined()); - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes, @@ -232,7 +232,7 @@ public abstract class AbstractBytesToBytesMapSuite { getByteArray(loc.getValueBase(), loc.getValueOffset(), recordLengthBytes)); try { - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes, @@ -260,7 +260,7 @@ public abstract class AbstractBytesToBytesMapSuite { Assert.assertFalse(loc.isDefined()); // Ensure that we store some zero-length keys if (i % 5 == 0) { - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( null, Platform.LONG_ARRAY_OFFSET, 0, @@ -269,7 +269,7 @@ public abstract class AbstractBytesToBytesMapSuite { 8 )); } else { - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( value, Platform.LONG_ARRAY_OFFSET, 8, @@ -349,7 +349,7 @@ public abstract class AbstractBytesToBytesMapSuite { KEY_LENGTH ); Assert.assertFalse(loc.isDefined()); - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( key, Platform.LONG_ARRAY_OFFSET, KEY_LENGTH, @@ -417,7 +417,7 @@ public abstract class AbstractBytesToBytesMapSuite { key.length ); Assert.assertFalse(loc.isDefined()); - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( key, Platform.BYTE_ARRAY_OFFSET, key.length, @@ -471,7 +471,7 @@ public abstract class AbstractBytesToBytesMapSuite { key.length ); Assert.assertFalse(loc.isDefined()); - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( key, Platform.BYTE_ARRAY_OFFSET, key.length, @@ -514,7 +514,7 @@ public abstract class AbstractBytesToBytesMapSuite { final BytesToBytesMap.Location loc = map.lookup(emptyArray, Platform.LONG_ARRAY_OFFSET, 0); Assert.assertFalse(loc.isDefined()); - Assert.assertFalse(loc.putNewKey( + Assert.assertFalse(loc.append( emptyArray, Platform.LONG_ARRAY_OFFSET, 0, emptyArray, Platform.LONG_ARRAY_OFFSET, 0)); } finally { map.free(); @@ -535,7 +535,7 @@ public abstract class AbstractBytesToBytesMapSuite { final long[] arr = new long[]{i}; final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8); success = - loc.putNewKey(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); if (!success) { break; } @@ -556,7 +556,7 @@ public abstract class AbstractBytesToBytesMapSuite { 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.putNewKey(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); } BytesToBytesMap.MapIterator iter = map.iterator(); for (i = 0; i < 100; i++) { @@ -587,6 +587,44 @@ public abstract class AbstractBytesToBytesMapSuite { } @Test + public void multipleValuesForSameKey() { + BytesToBytesMap map = + new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.75, 1024, false); + try { + int i; + for (i = 0; i < 1024; i++) { + final long[] arr = new long[]{i}; + map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8) + .append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + } + assert map.numKeys() == 1024; + assert map.numValues() == 1024; + for (i = 0; i < 1024; i++) { + final long[] arr = new long[]{i}; + map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8) + .append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + } + assert map.numKeys() == 1024; + assert map.numValues() == 2048; + for (i = 0; i < 1024; i++) { + final long[] arr = new long[]{i}; + final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8); + assert loc.isDefined(); + assert loc.nextValue(); + assert !loc.nextValue(); + } + BytesToBytesMap.MapIterator iter = map.iterator(); + for (i = 0; i < 2048; i++) { + assert iter.hasNext(); + final BytesToBytesMap.Location loc = iter.next(); + assert loc.isDefined(); + } + } finally { + map.free(); + } + } + + @Test public void initialCapacityBoundsChecking() { try { new BytesToBytesMap(taskMemoryManager, 0, PAGE_SIZE_BYTES); @@ -608,7 +646,7 @@ public abstract class AbstractBytesToBytesMapSuite { @Test public void testPeakMemoryUsed() { - final long recordLengthBytes = 24; + final long recordLengthBytes = 32; final long pageSizeBytes = 256 + 8; // 8 bytes for end-of-page marker final long numRecordsPerPage = (pageSizeBytes - 8) / recordLengthBytes; final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, 1024, pageSizeBytes); @@ -622,7 +660,7 @@ public abstract class AbstractBytesToBytesMapSuite { try { for (long i = 0; i < numRecordsPerPage * 10; i++) { final long[] value = new long[]{i}; - map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).putNewKey( + map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).append( value, Platform.LONG_ARRAY_OFFSET, 8, diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index 5bbb4ceb97..1a13233133 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -2,6 +2,9 @@ "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917391398, + "lastUpdatedEpoch" : 0, "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:11.398GMT", "lastUpdated" : "", @@ -14,6 +17,9 @@ "name" : "Spark shell", "attempts" : [ { "attemptId" : "2", + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950, + "lastUpdatedEpoch" : 0, "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:00.950GMT", "lastUpdated" : "", @@ -22,6 +28,9 @@ "completed" : true }, { "attemptId" : "1", + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890, + "lastUpdatedEpoch" : 0, "startTime" : "2015-05-06T13:03:00.880GMT", "endTime" : "2015-05-06T13:03:00.890GMT", "lastUpdated" : "", @@ -34,6 +43,9 @@ "name" : "Spark shell", "attempts" : [ { "attemptId" : "2", + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177, + "lastUpdatedEpoch" : 0, "startTime" : "2015-03-17T23:11:50.242GMT", "endTime" : "2015-03-17T23:12:25.177GMT", "lastUpdated" : "", @@ -42,6 +54,9 @@ "completed" : true }, { "attemptId" : "1", + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177, + "lastUpdatedEpoch" : 0, "startTime" : "2015-03-16T19:25:10.242GMT", "endTime" : "2015-03-16T19:25:45.177GMT", "lastUpdated" : "", @@ -53,6 +68,9 @@ "id" : "local-1425081759269", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1425081758277, + "endTimeEpoch" : 1425081766912, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-28T00:02:38.277GMT", "endTime" : "2015-02-28T00:02:46.912GMT", "lastUpdated" : "", @@ -64,6 +82,9 @@ "id" : "local-1422981780767", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1422981779720, + "endTimeEpoch" : 1422981788731, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-03T16:42:59.720GMT", "endTime" : "2015-02-03T16:43:08.731GMT", "lastUpdated" : "", @@ -75,6 +96,9 @@ "id" : "local-1422981759269", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1422981758277, + "endTimeEpoch" : 1422981766912, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-03T16:42:38.277GMT", "endTime" : "2015-02-03T16:42:46.912GMT", "lastUpdated" : "", diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index 5bbb4ceb97..1a13233133 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -2,6 +2,9 @@ "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917391398, + "lastUpdatedEpoch" : 0, "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:11.398GMT", "lastUpdated" : "", @@ -14,6 +17,9 @@ "name" : "Spark shell", "attempts" : [ { "attemptId" : "2", + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950, + "lastUpdatedEpoch" : 0, "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:00.950GMT", "lastUpdated" : "", @@ -22,6 +28,9 @@ "completed" : true }, { "attemptId" : "1", + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890, + "lastUpdatedEpoch" : 0, "startTime" : "2015-05-06T13:03:00.880GMT", "endTime" : "2015-05-06T13:03:00.890GMT", "lastUpdated" : "", @@ -34,6 +43,9 @@ "name" : "Spark shell", "attempts" : [ { "attemptId" : "2", + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177, + "lastUpdatedEpoch" : 0, "startTime" : "2015-03-17T23:11:50.242GMT", "endTime" : "2015-03-17T23:12:25.177GMT", "lastUpdated" : "", @@ -42,6 +54,9 @@ "completed" : true }, { "attemptId" : "1", + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177, + "lastUpdatedEpoch" : 0, "startTime" : "2015-03-16T19:25:10.242GMT", "endTime" : "2015-03-16T19:25:45.177GMT", "lastUpdated" : "", @@ -53,6 +68,9 @@ "id" : "local-1425081759269", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1425081758277, + "endTimeEpoch" : 1425081766912, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-28T00:02:38.277GMT", "endTime" : "2015-02-28T00:02:46.912GMT", "lastUpdated" : "", @@ -64,6 +82,9 @@ "id" : "local-1422981780767", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1422981779720, + "endTimeEpoch" : 1422981788731, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-03T16:42:59.720GMT", "endTime" : "2015-02-03T16:43:08.731GMT", "lastUpdated" : "", @@ -75,6 +96,9 @@ "id" : "local-1422981759269", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1422981758277, + "endTimeEpoch" : 1422981766912, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-03T16:42:38.277GMT", "endTime" : "2015-02-03T16:42:46.912GMT", "lastUpdated" : "", diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index 4a88eeee74..efc865919b 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -2,8 +2,8 @@ "id" : "<driver>", "hostPort" : "localhost:57971", "isActive" : true, - "rddBlocks" : 8, - "memoryUsed" : 28000128, + "rddBlocks" : 0, + "memoryUsed" : 0, "diskUsed" : 0, "totalCores" : 0, "maxTasks" : 0, diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json index 3f80a529a0..eacf04b901 100644 --- a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json @@ -2,6 +2,9 @@ "id" : "local-1422981759269", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1422981758277, + "endTimeEpoch" : 1422981766912, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-03T16:42:38.277GMT", "endTime" : "2015-02-03T16:42:46.912GMT", "lastUpdated" : "", diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json index 508bdc17ef..adad25bf17 100644 --- a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json @@ -2,6 +2,9 @@ "id" : "local-1422981780767", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1422981779720, + "endTimeEpoch" : 1422981788731, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-03T16:42:59.720GMT", "endTime" : "2015-02-03T16:43:08.731GMT", "lastUpdated" : "", @@ -13,6 +16,9 @@ "id" : "local-1422981759269", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1422981758277, + "endTimeEpoch" : 1422981766912, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-03T16:42:38.277GMT", "endTime" : "2015-02-03T16:42:46.912GMT", "lastUpdated" : "", diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index 5dca7d73de..a658909088 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -2,6 +2,9 @@ "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917391398, + "lastUpdatedEpoch" : 0, "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:11.398GMT", "lastUpdated" : "", @@ -14,6 +17,9 @@ "name" : "Spark shell", "attempts" : [ { "attemptId" : "2", + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950, + "lastUpdatedEpoch" : 0, "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:00.950GMT", "lastUpdated" : "", @@ -22,6 +28,9 @@ "completed" : true }, { "attemptId" : "1", + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890, + "lastUpdatedEpoch" : 0, "startTime" : "2015-05-06T13:03:00.880GMT", "endTime" : "2015-05-06T13:03:00.890GMT", "lastUpdated" : "", @@ -34,6 +43,9 @@ "name" : "Spark shell", "attempts" : [ { "attemptId" : "2", + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177, + "lastUpdatedEpoch" : 0, "startTime" : "2015-03-17T23:11:50.242GMT", "endTime" : "2015-03-17T23:12:25.177GMT", "lastUpdated" : "", @@ -42,6 +54,9 @@ "completed" : true }, { "attemptId" : "1", + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177, + "lastUpdatedEpoch" : 0, "startTime" : "2015-03-16T19:25:10.242GMT", "endTime" : "2015-03-16T19:25:45.177GMT", "lastUpdated" : "", @@ -54,6 +69,9 @@ "name": "Spark shell", "attempts": [ { + "startTimeEpoch" : 1425081758277, + "endTimeEpoch" : 1425081766912, + "lastUpdatedEpoch" : 0, "startTime": "2015-02-28T00:02:38.277GMT", "endTime": "2015-02-28T00:02:46.912GMT", "lastUpdated" : "", diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json index cca32c7910..0217facad9 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json @@ -2,6 +2,9 @@ "id" : "local-1422981780767", "name" : "Spark shell", "attempts" : [ { + "startTimeEpoch" : 1422981779720, + "endTimeEpoch" : 1422981788731, + "lastUpdatedEpoch" : 0, "startTime" : "2015-02-03T16:42:59.720GMT", "endTime" : "2015-02-03T16:43:08.731GMT", "lastUpdated" : "", diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json index 1ea1779e83..b20a26648e 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json @@ -3,6 +3,9 @@ "name" : "Spark shell", "attempts" : [ { "attemptId" : "2", + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177, + "lastUpdatedEpoch" : 0, "startTime" : "2015-03-17T23:11:50.242GMT", "endTime" : "2015-03-17T23:12:25.177GMT", "lastUpdated" : "", @@ -11,6 +14,9 @@ "completed" : true }, { "attemptId" : "1", + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177, + "lastUpdatedEpoch" : 0, "startTime" : "2015-03-16T19:25:10.242GMT", "endTime" : "2015-03-16T19:25:45.177GMT", "lastUpdated" : "", diff --git a/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json index f79a31022d..8878e547a7 100644 --- a/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json @@ -1,9 +1 @@ -[ { - "id" : 0, - "name" : "0", - "numPartitions" : 8, - "numCachedPartitions" : 8, - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 28000128, - "diskUsed" : 0 -} ]
\ No newline at end of file +[ ]
\ No newline at end of file diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index a54d27de91..fb9d9851cb 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -33,5 +33,4 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%t: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN -org.spark-project.jetty.LEVEL=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index ec192a8543..37879d11ca 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark +import java.util.Properties import java.util.concurrent.Semaphore import javax.annotation.concurrent.GuardedBy @@ -292,7 +293,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex dummyTask, mutable.HashMap(), mutable.HashMap(), serInstance) // Now we're on the executors. // Deserialize the task and assert that its accumulators are zero'ed out. - val (_, _, taskBytes) = Task.deserializeWithDependencies(taskSer) + val (_, _, _, taskBytes) = Task.deserializeWithDependencies(taskSer) val taskDeser = serInstance.deserialize[DummyTask]( taskBytes, Thread.currentThread.getContextClassLoader) // Assert that executors see only zeros @@ -403,6 +404,6 @@ private class SaveInfoListener extends SparkListener { private[spark] class DummyTask( val internalAccums: Seq[Accumulator[_]], val externalAccums: Seq[Accumulator[_]]) - extends Task[Int](0, 0, 0, internalAccums) { + extends Task[Int](0, 0, 0, internalAccums, new Properties) { override def runTask(c: TaskContext): Int = 1 } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 3dded4d486..2110d3d770 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -198,8 +198,8 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex blockManager.master.getLocations(blockId).foreach { cmId => val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId, blockId.toString) - val deserialized = serializerManager.dataDeserialize[Int](blockId, - new ChunkedByteBuffer(bytes.nioByteBuffer())).toList + val deserialized = serializerManager.dataDeserializeStream[Int](blockId, + new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream()).toList assert(deserialized === (1 to 100).toList) } } @@ -320,7 +320,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex Thread.sleep(200) } } catch { - case _: Throwable => { Thread.sleep(10) } + case _: Throwable => Thread.sleep(10) // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 80a1de6065..ee6b991461 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -928,8 +928,8 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { numTasks: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty ): StageInfo = { - new StageInfo( - stageId, 0, "name", numTasks, Seq.empty, Seq.empty, "no details", taskLocalityPreferences) + new StageInfo(stageId, 0, "name", numTasks, Seq.empty, Seq.empty, "no details", + Seq.empty, taskLocalityPreferences) } private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = { diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 3777d77f8f..713d5e58b4 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -174,9 +174,9 @@ class HeartbeatReceiverSuite val dummyExecutorEndpoint2 = new FakeExecutorEndpoint(rpcEnv) val dummyExecutorEndpointRef1 = rpcEnv.setupEndpoint("fake-executor-1", dummyExecutorEndpoint1) val dummyExecutorEndpointRef2 = rpcEnv.setupEndpoint("fake-executor-2", dummyExecutorEndpoint2) - fakeSchedulerBackend.driverEndpoint.askWithRetry[RegisterExecutorResponse]( + fakeSchedulerBackend.driverEndpoint.askWithRetry[Boolean]( RegisterExecutor(executorId1, dummyExecutorEndpointRef1, 0, Map.empty)) - fakeSchedulerBackend.driverEndpoint.askWithRetry[RegisterExecutorResponse]( + fakeSchedulerBackend.driverEndpoint.askWithRetry[Boolean]( RegisterExecutor(executorId2, dummyExecutorEndpointRef2, 0, Map.empty)) heartbeatReceiverRef.askWithRetry[Boolean](TaskSchedulerIsSet) addExecutorAndVerify(executorId1) @@ -255,7 +255,12 @@ class HeartbeatReceiverSuite /** * Dummy RPC endpoint to simulate executors. */ -private class FakeExecutorEndpoint(override val rpcEnv: RpcEnv) extends RpcEndpoint +private class FakeExecutorEndpoint(override val rpcEnv: RpcEnv) extends RpcEndpoint { + + override def receive: PartialFunction[Any, Unit] = { + case _ => + } +} /** * Dummy scheduler backend to simulate executor allocation requests to the cluster manager. diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 6ffa1c8ac1..cd7d2e1570 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark +import java.util.Properties import java.util.concurrent.{Callable, CyclicBarrier, Executors, ExecutorService} import org.scalatest.Matchers @@ -335,16 +336,16 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // first attempt -- its successful val writer1 = manager.getWriter[Int, Int](shuffleHandle, 0, - new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, metricsSystem, - InternalAccumulator.create(sc))) + new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem, + InternalAccumulator.createAll(sc))) val data1 = (1 to 10).map { x => x -> x} // second attempt -- also successful. We'll write out different data, // just to simulate the fact that the records may get written differently // depending on what gets spilled, what gets combined, etc. val writer2 = manager.getWriter[Int, Int](shuffleHandle, 0, - new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, metricsSystem, - InternalAccumulator.create(sc))) + new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem, + InternalAccumulator.createAll(sc))) val data2 = (11 to 20).map { x => x -> x} // interleave writes of both attempts -- we want to test that both attempts can occur @@ -372,8 +373,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC } val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1, - new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, metricsSystem, - InternalAccumulator.create(sc))) + new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem, + InternalAccumulator.createAll(sc))) val readData = reader.read().toIndexedSeq assert(readData === data1.toIndexedSeq || readData === data2.toIndexedSeq) diff --git a/core/src/test/scala/org/apache/spark/Smuggle.scala b/core/src/test/scala/org/apache/spark/Smuggle.scala index 9f0a1b4c25..9d9217ea1b 100644 --- a/core/src/test/scala/org/apache/spark/Smuggle.scala +++ b/core/src/test/scala/org/apache/spark/Smuggle.scala @@ -24,16 +24,16 @@ import scala.collection.mutable import scala.language.implicitConversions /** - * Utility wrapper to "smuggle" objects into tasks while bypassing serialization. - * This is intended for testing purposes, primarily to make locks, semaphores, and - * other constructs that would not survive serialization available from within tasks. - * A Smuggle reference is itself serializable, but after being serialized and - * deserialized, it still refers to the same underlying "smuggled" object, as long - * as it was deserialized within the same JVM. This can be useful for tests that - * depend on the timing of task completion to be deterministic, since one can "smuggle" - * a lock or semaphore into the task, and then the task can block until the test gives - * the go-ahead to proceed via the lock. - */ + * Utility wrapper to "smuggle" objects into tasks while bypassing serialization. + * This is intended for testing purposes, primarily to make locks, semaphores, and + * other constructs that would not survive serialization available from within tasks. + * A Smuggle reference is itself serializable, but after being serialized and + * deserialized, it still refers to the same underlying "smuggled" object, as long + * as it was deserialized within the same JVM. This can be useful for tests that + * depend on the timing of task completion to be deterministic, since one can "smuggle" + * a lock or semaphore into the task, and then the task can block until the test gives + * the go-ahead to proceed via the lock. + */ class Smuggle[T] private(val key: Symbol) extends Serializable { def smuggledObject: T = Smuggle.get(key) } @@ -41,13 +41,13 @@ class Smuggle[T] private(val key: Symbol) extends Serializable { object Smuggle { /** - * Wraps the specified object to be smuggled into a serialized task without - * being serialized itself. - * - * @param smuggledObject - * @tparam T - * @return Smuggle wrapper around smuggledObject. - */ + * Wraps the specified object to be smuggled into a serialized task without + * being serialized itself. + * + * @param smuggledObject + * @tparam T + * @return Smuggle wrapper around smuggledObject. + */ def apply[T](smuggledObject: T): Smuggle[T] = { val key = Symbol(UUID.randomUUID().toString) lock.writeLock().lock() @@ -72,12 +72,12 @@ object Smuggle { } /** - * Implicit conversion of a Smuggle wrapper to the object being smuggled. - * - * @param smuggle the wrapper to unpack. - * @tparam T - * @return the smuggled object represented by the wrapper. - */ + * Implicit conversion of a Smuggle wrapper to the object being smuggled. + * + * @param smuggle the wrapper to unpack. + * @tparam T + * @return the smuggled object represented by the wrapper. + */ implicit def unpackSmuggledObject[T](smuggle : Smuggle[T]): T = smuggle.smuggledObject } diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 3706455c3f..8feb3dee05 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -82,20 +82,18 @@ package object testPackage extends Assertions { val curCallSite = sc.getCallSite().shortForm // note: 2 lines after definition of "rdd" val rddCreationLine = rddCreationSite match { - case CALL_SITE_REGEX(func, file, line) => { + case CALL_SITE_REGEX(func, file, line) => assert(func === "makeRDD") assert(file === "SparkContextInfoSuite.scala") line.toInt - } case _ => fail("Did not match expected call site format") } curCallSite match { - case CALL_SITE_REGEX(func, file, line) => { + case CALL_SITE_REGEX(func, file, line) => assert(func === "getCallSite") // this is correct because we called it from outside of Spark assert(file === "SparkContextInfoSuite.scala") assert(line.toInt === rddCreationLine.toInt + 2) - } case _ => fail("Did not match expected call site format") } } diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala index f7a13ab399..09e21646ee 100644 --- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala +++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala @@ -35,7 +35,7 @@ class UnpersistSuite extends SparkFunSuite with LocalSparkContext { Thread.sleep(200) } } catch { - case _: Throwable => { Thread.sleep(10) } + case _: Throwable => Thread.sleep(10) // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } 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 96cb4fd0eb..2718976992 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -199,21 +199,21 @@ class SparkSubmitSuite val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) val childArgsStr = childArgs.mkString(" ") childArgsStr should include ("--class org.SomeClass") - childArgsStr should include ("--executor-memory 5g") - childArgsStr should include ("--driver-memory 4g") - childArgsStr should include ("--executor-cores 5") childArgsStr should include ("--arg arg1 --arg arg2") - childArgsStr should include ("--queue thequeue") childArgsStr should include regex ("--jar .*thejar.jar") - childArgsStr should include regex ("--addJars .*one.jar,.*two.jar,.*three.jar") - childArgsStr should include regex ("--files .*file1.txt,.*file2.txt") - childArgsStr should include regex ("--archives .*archive1.txt,.*archive2.txt") mainClass should be ("org.apache.spark.deploy.yarn.Client") classpath should have length (0) + + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.driver.memory") should be ("4g") + sysProps("spark.executor.cores") should be ("5") + sysProps("spark.yarn.queue") should be ("thequeue") + sysProps("spark.yarn.dist.jars") should include regex (".*one.jar,.*two.jar,.*three.jar") + sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt") + sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") sysProps("spark.app.name") should be ("beauty") sysProps("spark.ui.enabled") should be ("false") sysProps("SPARK_SUBMIT") should be ("true") - sysProps.keys should not contain ("spark.jars") } test("handles YARN client mode") { @@ -249,7 +249,8 @@ class SparkSubmitSuite sysProps("spark.executor.instances") should be ("6") sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt") sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") - sysProps("spark.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") + sysProps("spark.yarn.dist.jars") should include + regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") sysProps("SPARK_SUBMIT") should be ("true") sysProps("spark.ui.enabled") should be ("false") } 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 d2e24912b5..3d39bd4a74 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -561,7 +561,7 @@ class StandaloneDynamicAllocationSuite when(endpointRef.address).thenReturn(mockAddress) val message = RegisterExecutor(id, endpointRef, 10, Map.empty) val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] - backend.driverEndpoint.askWithRetry[CoarseGrainedClusterMessage](message) + backend.driverEndpoint.askWithRetry[Boolean](message) } } 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 5822261d8d..2a013aca7b 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 @@ -140,8 +140,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "stage task list from multi-attempt app json(2)" -> "applications/local-1426533911241/2/stages/0/0/taskList", - "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", - "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" + "rdd list storage json" -> "applications/local-1422981780767/storage/rdd" + // Todo: enable this test when logging the even of onBlockUpdated. See: SPARK-13845 + // "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" ) // run a bunch of characterization tests -- just verify the behavior is the same as what is saved @@ -161,7 +162,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val json = if (jsonOrg.indexOf("lastUpdated") >= 0) { val subStrings = jsonOrg.split(",") for (i <- subStrings.indices) { - if (subStrings(i).indexOf("lastUpdated") >= 0) { + if (subStrings(i).indexOf("lastUpdatedEpoch") >= 0) { + subStrings(i) = subStrings(i).replaceAll("(\\d+)", "0") + } else if (subStrings(i).indexOf("lastUpdated") >= 0) { subStrings(i) = "\"lastUpdated\":\"\"" } } diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 088b05403c..d91f50f18f 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -285,8 +285,8 @@ class TaskMetricsSuite extends SparkFunSuite { // set and increment values in.setBytesRead(1L) in.setBytesRead(2L) - in.incRecordsReadInternal(1L) - in.incRecordsReadInternal(2L) + in.incRecordsRead(1L) + in.incRecordsRead(2L) in.setReadMethod(DataReadMethod.Disk) // assert new values exist assertValEquals(_.bytesRead, BYTES_READ, 2L) diff --git a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala index 0644148eae..337fd7e85e 100644 --- a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala @@ -26,7 +26,7 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: int") { val conf = new SparkConf() - val iConf = ConfigBuilder("spark.int").intConf.withDefault(1) + val iConf = ConfigBuilder("spark.int").intConf.createWithDefault(1) assert(conf.get(iConf) === 1) conf.set(iConf, 2) assert(conf.get(iConf) === 2) @@ -34,21 +34,21 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: long") { val conf = new SparkConf() - val lConf = ConfigBuilder("spark.long").longConf.withDefault(0L) + val lConf = ConfigBuilder("spark.long").longConf.createWithDefault(0L) conf.set(lConf, 1234L) assert(conf.get(lConf) === 1234L) } test("conf entry: double") { val conf = new SparkConf() - val dConf = ConfigBuilder("spark.double").doubleConf.withDefault(0.0) + val dConf = ConfigBuilder("spark.double").doubleConf.createWithDefault(0.0) conf.set(dConf, 20.0) assert(conf.get(dConf) === 20.0) } test("conf entry: boolean") { val conf = new SparkConf() - val bConf = ConfigBuilder("spark.boolean").booleanConf.withDefault(false) + val bConf = ConfigBuilder("spark.boolean").booleanConf.createWithDefault(false) assert(!conf.get(bConf)) conf.set(bConf, true) assert(conf.get(bConf)) @@ -56,7 +56,7 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: optional") { val conf = new SparkConf() - val optionalConf = ConfigBuilder("spark.optional").intConf.optional + val optionalConf = ConfigBuilder("spark.optional").intConf.createOptional assert(conf.get(optionalConf) === None) conf.set(optionalConf, 1) assert(conf.get(optionalConf) === Some(1)) @@ -64,7 +64,7 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: fallback") { val conf = new SparkConf() - val parentConf = ConfigBuilder("spark.int").intConf.withDefault(1) + val parentConf = ConfigBuilder("spark.int").intConf.createWithDefault(1) val confWithFallback = ConfigBuilder("spark.fallback").fallbackConf(parentConf) assert(conf.get(confWithFallback) === 1) conf.set(confWithFallback, 2) @@ -74,7 +74,7 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: time") { val conf = new SparkConf() - val time = ConfigBuilder("spark.time").timeConf(TimeUnit.SECONDS).withDefaultString("1h") + val time = ConfigBuilder("spark.time").timeConf(TimeUnit.SECONDS).createWithDefaultString("1h") assert(conf.get(time) === 3600L) conf.set(time.key, "1m") assert(conf.get(time) === 60L) @@ -82,7 +82,7 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: bytes") { val conf = new SparkConf() - val bytes = ConfigBuilder("spark.bytes").bytesConf(ByteUnit.KiB).withDefaultString("1m") + val bytes = ConfigBuilder("spark.bytes").bytesConf(ByteUnit.KiB).createWithDefaultString("1m") assert(conf.get(bytes) === 1024L) conf.set(bytes.key, "1k") assert(conf.get(bytes) === 1L) @@ -90,7 +90,7 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: string seq") { val conf = new SparkConf() - val seq = ConfigBuilder("spark.seq").stringConf.toSequence.withDefault(Seq()) + val seq = ConfigBuilder("spark.seq").stringConf.toSequence.createWithDefault(Seq()) conf.set(seq.key, "1,,2, 3 , , 4") assert(conf.get(seq) === Seq("1", "2", "3", "4")) conf.set(seq, Seq("1", "2")) @@ -99,7 +99,7 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: int seq") { val conf = new SparkConf() - val seq = ConfigBuilder("spark.seq").intConf.toSequence.withDefault(Seq()) + val seq = ConfigBuilder("spark.seq").intConf.toSequence.createWithDefault(Seq()) conf.set(seq.key, "1,,2, 3 , , 4") assert(conf.get(seq) === Seq(1, 2, 3, 4)) conf.set(seq, Seq(1, 2)) @@ -111,7 +111,7 @@ class ConfigEntrySuite extends SparkFunSuite { val transformationConf = ConfigBuilder("spark.transformation") .stringConf .transform(_.toLowerCase()) - .withDefault("FOO") + .createWithDefault("FOO") assert(conf.get(transformationConf) === "foo") conf.set(transformationConf, "BAR") @@ -123,7 +123,7 @@ class ConfigEntrySuite extends SparkFunSuite { val enum = ConfigBuilder("spark.enum") .stringConf .checkValues(Set("a", "b", "c")) - .withDefault("a") + .createWithDefault("a") assert(conf.get(enum) === "a") conf.set(enum, "b") @@ -138,7 +138,7 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: conversion error") { val conf = new SparkConf() - val conversionTest = ConfigBuilder("spark.conversionTest").doubleConf.optional + val conversionTest = ConfigBuilder("spark.conversionTest").doubleConf.createOptional conf.set(conversionTest.key, "abc") val conversionError = intercept[IllegalArgumentException] { conf.get(conversionTest) @@ -148,7 +148,7 @@ class ConfigEntrySuite extends SparkFunSuite { test("default value handling is null-safe") { val conf = new SparkConf() - val stringConf = ConfigBuilder("spark.string").stringConf.withDefault(null) + val stringConf = ConfigBuilder("spark.string").stringConf.createWithDefault(null) assert(conf.get(stringConf) === null) } diff --git a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala index aab70e7431..f205d4f0d6 100644 --- a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala @@ -52,7 +52,7 @@ class ChunkedByteBufferSuite extends SparkFunSuite { test("copy() does not affect original buffer's position") { val chunkedByteBuffer = new ChunkedByteBuffer(Array(ByteBuffer.allocate(8))) - chunkedByteBuffer.copy() + chunkedByteBuffer.copy(ByteBuffer.allocate) assert(chunkedByteBuffer.getChunks().head.position() === 0) } diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index aaca653c58..99d5b496bc 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -71,24 +71,25 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft */ protected def makeMemoryStore(mm: MemoryManager): MemoryStore = { val ms = mock(classOf[MemoryStore], RETURNS_SMART_NULLS) - when(ms.evictBlocksToFreeSpace(any(), anyLong())).thenAnswer(evictBlocksToFreeSpaceAnswer(mm)) + when(ms.evictBlocksToFreeSpace(any(), anyLong(), any())) + .thenAnswer(evictBlocksToFreeSpaceAnswer(mm)) mm.setMemoryStore(ms) ms } /** - * Simulate the part of [[MemoryStore.evictBlocksToFreeSpace]] that releases storage memory. - * - * This is a significant simplification of the real method, which actually drops existing - * blocks based on the size of each block. Instead, here we simply release as many bytes - * as needed to ensure the requested amount of free space. This allows us to set up the - * test without relying on the [[org.apache.spark.storage.BlockManager]], which brings in - * many other dependencies. - * - * Every call to this method will set a global variable, [[evictBlocksToFreeSpaceCalled]], that - * records the number of bytes this is called with. This variable is expected to be cleared - * by the test code later through [[assertEvictBlocksToFreeSpaceCalled]]. - */ + * Simulate the part of [[MemoryStore.evictBlocksToFreeSpace]] that releases storage memory. + * + * This is a significant simplification of the real method, which actually drops existing + * blocks based on the size of each block. Instead, here we simply release as many bytes + * as needed to ensure the requested amount of free space. This allows us to set up the + * test without relying on the [[org.apache.spark.storage.BlockManager]], which brings in + * many other dependencies. + * + * Every call to this method will set a global variable, [[evictBlocksToFreeSpaceCalled]], that + * records the number of bytes this is called with. This variable is expected to be cleared + * by the test code later through [[assertEvictBlocksToFreeSpaceCalled]]. + */ private def evictBlocksToFreeSpaceAnswer(mm: MemoryManager): Answer[Long] = { new Answer[Long] { override def answer(invocation: InvocationOnMock): Long = { diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala index 2b5e4b80e9..362cd861cc 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.memory +import java.util.Properties + import org.apache.spark.{SparkEnv, TaskContext, TaskContextImpl} /** @@ -31,6 +33,7 @@ object MemoryTestingUtils { taskAttemptId = 0, attemptNumber = 0, taskMemoryManager = taskMemoryManager, + localProperties = new Properties, metricsSystem = env.metricsSystem) } } diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 6da18cfd49..ed15e77ff1 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -108,11 +108,11 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) val securityManager0 = new SecurityManager(conf0) - val exec0 = new NettyBlockTransferService(conf0, securityManager0, numCores = 1) + val exec0 = new NettyBlockTransferService(conf0, securityManager0, "localhost", numCores = 1) exec0.init(blockManager) val securityManager1 = new SecurityManager(conf1) - val exec1 = new NettyBlockTransferService(conf1, securityManager1, numCores = 1) + val exec1 = new NettyBlockTransferService(conf1, securityManager1, "localhost", numCores = 1) exec1.init(blockManager) val result = fetchBlock(exec0, exec1, "1", blockId) match { diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala index cc1a9e0287..f3c156e4f7 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala @@ -80,7 +80,7 @@ class NettyBlockTransferServiceSuite .set("spark.blockManager.port", port.toString) val securityManager = new SecurityManager(conf) val blockDataManager = mock(classOf[BlockDataManager]) - val service = new NettyBlockTransferService(conf, securityManager, numCores = 1) + val service = new NettyBlockTransferService(conf, securityManager, "localhost", numCores = 1) service.init(blockDataManager) service } diff --git a/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala b/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala new file mode 100644 index 0000000000..a79f5b4d74 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala @@ -0,0 +1,107 @@ +/* + * 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.partial + +import org.apache.spark._ +import org.apache.spark.util.StatCounter + +class SumEvaluatorSuite extends SparkFunSuite with SharedSparkContext { + + test("correct handling of count 1") { + + // setup + val counter = new StatCounter(List(2.0)) + // count of 10 because it's larger than 1, + // and 0.95 because that's the default + val evaluator = new SumEvaluator(10, 0.95) + // arbitrarily assign id 1 + evaluator.merge(1, counter) + + // execute + val res = evaluator.currentResult() + // 38.0 - 7.1E-15 because that's how the maths shakes out + val targetMean = 38.0 - 7.1E-15 + + // Sanity check that equality works on BoundedDouble + assert(new BoundedDouble(2.0, 0.95, 1.1, 1.2) == new BoundedDouble(2.0, 0.95, 1.1, 1.2)) + + // actual test + assert(res == + new BoundedDouble(targetMean, 0.950, Double.NegativeInfinity, Double.PositiveInfinity)) + } + + test("correct handling of count 0") { + + // setup + val counter = new StatCounter(List()) + // count of 10 because it's larger than 0, + // and 0.95 because that's the default + val evaluator = new SumEvaluator(10, 0.95) + // arbitrarily assign id 1 + evaluator.merge(1, counter) + + // execute + val res = evaluator.currentResult() + // assert + assert(res == new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity)) + } + + test("correct handling of NaN") { + + // setup + val counter = new StatCounter(List(1, Double.NaN, 2)) + // count of 10 because it's larger than 0, + // and 0.95 because that's the default + val evaluator = new SumEvaluator(10, 0.95) + // arbitrarily assign id 1 + evaluator.merge(1, counter) + + // execute + val res = evaluator.currentResult() + // assert - note semantics of == in face of NaN + assert(res.mean.isNaN) + assert(res.confidence == 0.95) + assert(res.low == Double.NegativeInfinity) + assert(res.high == Double.PositiveInfinity) + } + + test("correct handling of > 1 values") { + + // setup + val counter = new StatCounter(List(1, 3, 2)) + // count of 10 because it's larger than 0, + // and 0.95 because that's the default + val evaluator = new SumEvaluator(10, 0.95) + // arbitrarily assign id 1 + evaluator.merge(1, counter) + + // execute + val res = evaluator.currentResult() + + // These vals because that's how the maths shakes out + val targetMean = 78.0 + val targetLow = -117.617 + 2.732357258139473E-5 + val targetHigh = 273.617 - 2.7323572624027292E-5 + val target = new BoundedDouble(targetMean, 0.95, targetLow, targetHigh) + + + // check that values are within expected tolerance of expectation + assert(res == target) + } + +} diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala index 132a5fa9a8..cb0de1c6be 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala @@ -29,6 +29,8 @@ class MockSampler extends RandomSampler[Long, Long] { s = seed } + override def sample(): Int = 1 + override def sample(items: Iterator[Long]): Iterator[Long] = { Iterator(s) } diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index 43e61241b6..cebac2097f 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -127,9 +127,8 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { override val rpcEnv = env override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case msg: String => { + case msg: String => context.reply(msg) - } } }) val reply = rpcEndpointRef.askWithRetry[String]("hello") @@ -141,9 +140,8 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { override val rpcEnv = env override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case msg: String => { + case msg: String => context.reply(msg) - } } }) @@ -164,10 +162,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { override val rpcEnv = env override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case msg: String => { + case msg: String => Thread.sleep(100) context.reply(msg) - } } }) @@ -317,10 +314,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { override val rpcEnv = env override def receive: PartialFunction[Any, Unit] = { - case m => { + case m => self callSelfSuccessfully = true - } } }) @@ -682,9 +678,8 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { override val rpcEnv = localEnv override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case msg: String => { + case msg: String => context.reply(msg) - } } }) val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "ask-authentication") 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 2293c11dad..fd96fb04f8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1144,7 +1144,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou // SPARK-9809 -- this stage is submitted without a task for each partition (because some of // the shuffle map output is still available from stage 0); make sure we've still got internal // accumulators setup - assert(scheduler.stageIdToStage(2).internalAccumulators.nonEmpty) + assert(scheduler.stageIdToStage(2).latestInfo.internalAccumulators.nonEmpty) completeShuffleMapStageSuccessfully(2, 0, 2) completeNextResultStageWithSuccess(3, 1, idx => idx + 1234) assert(results === Map(0 -> 1234, 1 -> 1235)) 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 f7e16af9d3..e3e6df6831 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -17,12 +17,14 @@ package org.apache.spark.scheduler +import java.util.Properties + import org.apache.spark.TaskContext class FakeTask( stageId: Int, prefLocs: Seq[TaskLocation] = Nil) - extends Task[Int](stageId, 0, 0, Seq.empty) { + extends Task[Int](stageId, 0, 0, Seq.empty, new Properties) { override def runTask(context: TaskContext): Int = 0 override def preferredLocations: Seq[TaskLocation] = prefLocs } diff --git a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala index 1dca4bd89f..76a7087645 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.io.{IOException, ObjectInputStream, ObjectOutputStream} +import java.util.Properties import org.apache.spark.TaskContext @@ -25,7 +26,7 @@ import org.apache.spark.TaskContext * A Task implementation that fails to serialize. */ private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) - extends Task[Array[Byte]](stageId, 0, 0, Seq.empty) { + extends Task[Array[Byte]](stageId, 0, 0, Seq.empty, new Properties) { override def runTask(context: TaskContext): Array[Byte] = Array.empty[Byte] override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]() diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala index 9f41aca8a1..601f1c378c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala @@ -38,7 +38,7 @@ class OutputCommitCoordinatorIntegrationSuite super.beforeAll() val conf = new SparkConf() .set("master", "local[2,4]") - .set("spark.speculation", "true") + .set("spark.hadoop.outputCommitCoordination.enabled", "true") .set("spark.hadoop.mapred.output.committer.class", classOf[ThrowExceptionOnFirstAttemptOutputCommitter].getCanonicalName) sc = new SparkContext("local[2, 4]", "test", conf) diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index c461da65bd..8e509de767 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -77,7 +77,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { val conf = new SparkConf() .setMaster("local[4]") .setAppName(classOf[OutputCommitCoordinatorSuite].getSimpleName) - .set("spark.speculation", "true") + .set("spark.hadoop.outputCommitCoordination.enabled", "true") sc = new SparkContext(conf) { override private[spark] def createSparkEnv( conf: SparkConf, 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 58d217ffef..b854d742b5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.scalatest.Matchers -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkFunSuite} +import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.{ResetSystemProperties, RpcUtils} @@ -377,13 +377,18 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match } test("registering listeners via spark.extraListeners") { + val listeners = Seq( + classOf[ListenerThatAcceptsSparkConf], + classOf[FirehoseListenerThatAcceptsSparkConf], + classOf[BasicJobCounter]) val conf = new SparkConf().setMaster("local").setAppName("test") - .set("spark.extraListeners", classOf[ListenerThatAcceptsSparkConf].getName + "," + - classOf[BasicJobCounter].getName) + .set("spark.extraListeners", listeners.map(_.getName).mkString(",")) sc = new SparkContext(conf) sc.listenerBus.listeners.asScala.count(_.isInstanceOf[BasicJobCounter]) should be (1) sc.listenerBus.listeners.asScala .count(_.isInstanceOf[ListenerThatAcceptsSparkConf]) should be (1) + sc.listenerBus.listeners.asScala + .count(_.isInstanceOf[FirehoseListenerThatAcceptsSparkConf]) should be (1) } /** @@ -476,3 +481,11 @@ private class ListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkListene var count = 0 override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1 } + +private class FirehoseListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkFirehoseListener { + var count = 0 + override def onEvent(event: SparkListenerEvent): Unit = event match { + case job: SparkListenerJobEnd => count += 1 + case _ => + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 5df541e5a5..5ca0c6419d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -17,12 +17,14 @@ package org.apache.spark.scheduler +import java.util.Properties + import org.mockito.Matchers.any import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark._ -import org.apache.spark.executor.TaskMetricsSuite +import org.apache.spark.executor.{Executor, TaskMetricsSuite} import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.JvmSource import org.apache.spark.network.util.JavaUtils @@ -59,7 +61,8 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark val closureSerializer = SparkEnv.get.closureSerializer.newInstance() val func = (c: TaskContext, i: Iterator[String]) => i.next() val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func)))) - val task = new ResultTask[String, String](0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0) + val task = new ResultTask[String, String]( + 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties) intercept[RuntimeException] { task.run(0, 0, null) } @@ -80,7 +83,8 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark val closureSerializer = SparkEnv.get.closureSerializer.newInstance() val func = (c: TaskContext, i: Iterator[String]) => i.next() val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func)))) - val task = new ResultTask[String, String](0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0) + val task = new ResultTask[String, String]( + 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties) intercept[RuntimeException] { task.run(0, 0, null) } @@ -171,9 +175,10 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark val initialAccums = InternalAccumulator.createAll() // Create a dummy task. We won't end up running this; we just want to collect // accumulator updates from it. - val task = new Task[Int](0, 0, 0, Seq.empty[Accumulator[_]]) { + val task = new Task[Int](0, 0, 0, Seq.empty[Accumulator[_]], new Properties) { context = new TaskContextImpl(0, 0, 0L, 0, new TaskMemoryManager(SparkEnv.get.memoryManager, 0L), + new Properties, SparkEnv.get.metricsSystem, initialAccums) context.taskMetrics.registerAccumulator(acc1) @@ -190,6 +195,17 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark TaskMetricsSuite.assertUpdatesEquals(accumUpdates3, accumUpdates4) } + test("localProperties are propagated to executors correctly") { + sc = new SparkContext("local", "test") + sc.setLocalProperty("testPropKey", "testPropValue") + val res = sc.parallelize(Array(1), 1).map(i => i).map(i => { + val inTask = TaskContext.get().getLocalProperty("testPropKey") + val inDeser = Executor.taskDeserializationProps.get().getProperty("testPropKey") + s"$inTask,$inDeser" + }).collect() + assert(res === Array("testPropValue,testPropValue")) + } + } private object TaskContextSuite { 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 167d3fd2e4..ade8e84d84 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.util.Random +import java.util.{Properties, Random} import scala.collection.Map import scala.collection.mutable @@ -138,7 +138,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex /** * A Task implementation that results in a large serialized task. */ -class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0, Seq.empty) { +class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0, Seq.empty, new Properties) { val randomBuffer = new Array[Byte](TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024) val random = new Random(0) random.nextBytes(randomBuffer) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala index dbef6868f2..a32423dc4f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala @@ -136,4 +136,40 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi capture.capture() ) } + + test("escapes commandline args for the shell") { + val conf = new SparkConf() + conf.setMaster("mesos://localhost:5050") + conf.setAppName("spark mesos") + val scheduler = new MesosClusterScheduler( + new BlackHoleMesosClusterPersistenceEngineFactory, conf) { + override def start(): Unit = { ready = true } + } + val escape = scheduler.shellEscape _ + def wrapped(str: String): String = "\"" + str + "\"" + + // Wrapped in quotes + assert(escape("'should be left untouched'") === "'should be left untouched'") + assert(escape("\"should be left untouched\"") === "\"should be left untouched\"") + + // Harmless + assert(escape("") === "") + assert(escape("harmless") === "harmless") + assert(escape("har-m.l3ss") === "har-m.l3ss") + + // Special Chars escape + assert(escape("should escape this \" quote") === wrapped("should escape this \\\" quote")) + assert(escape("shouldescape\"quote") === wrapped("shouldescape\\\"quote")) + assert(escape("should escape this $ dollar") === wrapped("should escape this \\$ dollar")) + assert(escape("should escape this ` backtick") === wrapped("should escape this \\` backtick")) + assert(escape("""should escape this \ backslash""") + === wrapped("""should escape this \\ backslash""")) + assert(escape("""\"?""") === wrapped("""\\\"?""")) + + + // Special Chars no escape only wrap + List(" ", "'", "<", ">", "&", "|", "?", "*", ";", "!", "#", "(", ")").foreach(char => { + assert(escape(s"onlywrap${char}this") === wrapped(s"onlywrap${char}this")) + }) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala index 85437b2f80..ceb3a52983 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala @@ -59,10 +59,10 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("parse a non-empty constraint string correctly") { val expectedMap = Map( - "tachyon" -> Set("true"), + "os" -> Set("centos7"), "zone" -> Set("us-east-1a", "us-east-1b") ) - utils.parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b") should be (expectedMap) + utils.parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b") should be (expectedMap) } test("parse an empty constraint string correctly") { @@ -71,35 +71,35 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("throw an exception when the input is malformed") { an[IllegalArgumentException] should be thrownBy - utils.parseConstraintString("tachyon;zone:us-east") + utils.parseConstraintString("os;zone:us-east") } test("empty values for attributes' constraints matches all values") { - val constraintsStr = "tachyon:" + val constraintsStr = "os:" val parsedConstraints = utils.parseConstraintString(constraintsStr) - parsedConstraints shouldBe Map("tachyon" -> Set()) + parsedConstraints shouldBe Map("os" -> Set()) val zoneSet = Value.Set.newBuilder().addItem("us-east-1a").addItem("us-east-1b").build() - val noTachyonOffer = Map("zone" -> zoneSet) - val tachyonTrueOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build()) - val tachyonFalseOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("false").build()) + val noOsOffer = Map("zone" -> zoneSet) + val centosOffer = Map("os" -> Value.Text.newBuilder().setValue("centos").build()) + val ubuntuOffer = Map("os" -> Value.Text.newBuilder().setValue("ubuntu").build()) - utils.matchesAttributeRequirements(parsedConstraints, noTachyonOffer) shouldBe false - utils.matchesAttributeRequirements(parsedConstraints, tachyonTrueOffer) shouldBe true - utils.matchesAttributeRequirements(parsedConstraints, tachyonFalseOffer) shouldBe true + utils.matchesAttributeRequirements(parsedConstraints, noOsOffer) shouldBe false + utils.matchesAttributeRequirements(parsedConstraints, centosOffer) shouldBe true + utils.matchesAttributeRequirements(parsedConstraints, ubuntuOffer) shouldBe true } test("subset match is performed for set attributes") { val supersetConstraint = Map( - "tachyon" -> Value.Text.newBuilder().setValue("true").build(), + "os" -> Value.Text.newBuilder().setValue("ubuntu").build(), "zone" -> Value.Set.newBuilder() .addItem("us-east-1a") .addItem("us-east-1b") .addItem("us-east-1c") .build()) - val zoneConstraintStr = "tachyon:;zone:us-east-1a,us-east-1c" + val zoneConstraintStr = "os:;zone:us-east-1a,us-east-1c" val parsedConstraints = utils.parseConstraintString(zoneConstraintStr) utils.matchesAttributeRequirements(parsedConstraints, supersetConstraint) shouldBe true @@ -131,10 +131,10 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS } test("equality match is performed for text attributes") { - val offerAttribs = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build()) + val offerAttribs = Map("os" -> Value.Text.newBuilder().setValue("centos7").build()) - val trueConstraint = utils.parseConstraintString("tachyon:true") - val falseConstraint = utils.parseConstraintString("tachyon:false") + val trueConstraint = utils.parseConstraintString("os:centos7") + val falseConstraint = utils.parseConstraintString("os:ubuntu") utils.matchesAttributeRequirements(trueConstraint, offerAttribs) shouldBe true utils.matchesAttributeRequirements(falseConstraint, offerAttribs) shouldBe false diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 7ee76aa4c6..9d1bd7ec89 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.storage +import java.util.Properties + import scala.concurrent.{Await, ExecutionContext, Future} import scala.language.implicitConversions import scala.reflect.ClassTag @@ -58,7 +60,8 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { private def withTaskId[T](taskAttemptId: Long)(block: => T): T = { try { - TaskContext.setTaskContext(new TaskContextImpl(0, 0, taskAttemptId, 0, null, null)) + TaskContext.setTaskContext( + new TaskContextImpl(0, 0, taskAttemptId, 0, null, new Properties, null)) block } finally { TaskContext.unset() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 98e8450fa1..d26df7e760 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ import org.apache.spark._ -import org.apache.spark.memory.StaticMemoryManager +import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.rpc.RpcEnv @@ -60,8 +60,10 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo private def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { - val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1) - val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) + conf.set("spark.testing.memory", maxMem.toString) + conf.set("spark.memory.offHeap.size", maxMem.toString) + val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", numCores = 1) + val memManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(serializer, conf) val store = new BlockManager(name, rpcEnv, master, serializerManager, conf, memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) @@ -76,6 +78,9 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo conf.set("spark.authenticate", "false") conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set("spark.testing", "true") + conf.set("spark.memory.fraction", "1") + conf.set("spark.memory.storageFraction", "1") conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") @@ -172,6 +177,10 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo testReplication(5, storageLevels) } + test("block replication - off-heap") { + testReplication(2, Seq(OFF_HEAP, StorageLevel(true, true, true, false, 2))) + } + test("block replication - 2x replication without peers") { intercept[org.scalatest.exceptions.TestFailedException] { testReplication(1, @@ -262,7 +271,8 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo val failableTransfer = mock(classOf[BlockTransferService]) // this wont actually work when(failableTransfer.hostName).thenReturn("some-hostname") when(failableTransfer.port).thenReturn(1000) - val memManager = new StaticMemoryManager(conf, Long.MaxValue, 10000, numCores = 1) + conf.set("spark.testing.memory", "10000") + val memManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(serializer, conf) val failableStore = new BlockManager("failable-store", rpcEnv, master, serializerManager, conf, memManager, mapOutputTracker, shuffleManager, failableTransfer, securityMgr, 0) @@ -392,10 +402,14 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo // If the block is supposed to be in memory, then drop the copy of the block in // this store test whether master is updated with zero memory usage this store if (storageLevel.useMemory) { + val sl = if (storageLevel.useOffHeap) { + StorageLevel(false, true, true, false, 1) + } else { + MEMORY_ONLY_SER + } // Force the block to be dropped by adding a number of dummy blocks (1 to 10).foreach { - i => - testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), MEMORY_ONLY_SER) + i => testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), sl) } (1 to 10).foreach { i => testStore.removeBlock(s"dummy-block-$i") 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 6fc32cb30a..a1c2933584 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -34,7 +34,7 @@ import org.scalatest.concurrent.Timeouts._ import org.apache.spark._ import org.apache.spark.executor.DataReadMethod -import org.apache.spark.memory.{MemoryMode, StaticMemoryManager} +import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.{BlockDataManager, BlockTransferService} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.NettyBlockTransferService @@ -74,10 +74,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE name: String = SparkContext.DRIVER_IDENTIFIER, master: BlockManagerMaster = this.master, transferService: Option[BlockTransferService] = Option.empty): BlockManager = { + conf.set("spark.testing.memory", maxMem.toString) + conf.set("spark.memory.offHeap.size", maxMem.toString) val serializer = new KryoSerializer(conf) val transfer = transferService - .getOrElse(new NettyBlockTransferService(conf, securityMgr, numCores = 1)) - val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) + .getOrElse(new NettyBlockTransferService(conf, securityMgr, "localhost", numCores = 1)) + val memManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(serializer, conf) val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, conf, memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) @@ -92,6 +94,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE System.setProperty("os.arch", "amd64") conf = new SparkConf(false) .set("spark.app.id", "test") + .set("spark.testing", "true") + .set("spark.memory.fraction", "1") + .set("spark.memory.storageFraction", "1") .set("spark.kryoserializer.buffer", "1m") .set("spark.test.useCompressedOops", "true") .set("spark.storage.unrollFraction", "0.4") @@ -485,7 +490,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val blockManager = makeBlockManager(128, "exec", bmMaster) val getLocations = PrivateMethod[Seq[BlockManagerId]]('getLocations) val locations = blockManager invokePrivate getLocations(BroadcastBlockId(0)) - assert(locations.map(_.host) === Seq(localHost, localHost, otherHost)) + assert(locations.map(_.host).toSet === Set(localHost, localHost, otherHost)) } test("SPARK-9591: getRemoteBytes from another location when Exception throw") { @@ -510,6 +515,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } + test("SPARK-14252: getOrElseUpdate should still read from remote storage") { + store = makeBlockManager(8000, "executor1") + store2 = makeBlockManager(8000, "executor2") + val list1 = List(new Array[Byte](4000)) + store2.putIterator( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(store.getOrElseUpdate( + "list1", + StorageLevel.MEMORY_ONLY, + ClassTag.Any, + () => throw new AssertionError("attempted to compute locally")).isLeft) + } + test("in-memory LRU storage") { testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY) } @@ -518,6 +536,14 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY_SER) } + test("in-memory LRU storage with off-heap") { + testInMemoryLRUStorage(StorageLevel( + useDisk = false, + useMemory = true, + useOffHeap = true, + deserialized = false, replication = 1)) + } + private def testInMemoryLRUStorage(storageLevel: StorageLevel): Unit = { store = makeBlockManager(12000) val a1 = new Array[Byte](4000) @@ -608,6 +634,14 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, getAsBytes = true) } + test("disk and off-heap memory storage") { + testDiskAndMemoryStorage(StorageLevel.OFF_HEAP, getAsBytes = false) + } + + test("disk and off-heap memory storage with getLocalBytes") { + testDiskAndMemoryStorage(StorageLevel.OFF_HEAP, getAsBytes = true) + } + def testDiskAndMemoryStorage( storageLevel: StorageLevel, getAsBytes: Boolean): Unit = { @@ -817,12 +851,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("block store put failure") { // Use Java serializer so we can create an unserializable error. - val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1) - val memoryManager = new StaticMemoryManager( - conf, - maxOnHeapExecutionMemory = Long.MaxValue, - maxOnHeapStorageMemory = 1200, - numCores = 1) + conf.set("spark.testing.memory", "1200") + val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", numCores = 1) + val memoryManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, serializerManager, conf, memoryManager, mapOutputTracker, @@ -928,6 +959,16 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(!store.diskStore.contains("list3"), "list3 was in disk store") assert(!store.diskStore.contains("list4"), "list4 was in disk store") assert(!store.diskStore.contains("list5"), "list5 was in disk store") + + // remove block - list2 should be removed from disk + val updatedBlocks6 = getUpdatedBlocks { + store.removeBlock( + "list2", tellMaster = true) + } + assert(updatedBlocks6.size === 1) + assert(updatedBlocks6.head._1 === TestBlockId("list2")) + assert(updatedBlocks6.head._2.storageLevel == StorageLevel.NONE) + assert(!store.diskStore.contains("list2"), "list2 was in disk store") } test("query block statuses") { diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala index 43e832dc02..145d432afe 100644 --- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import org.scalatest._ import org.apache.spark._ -import org.apache.spark.memory.StaticMemoryManager +import org.apache.spark.memory.{MemoryMode, StaticMemoryManager} import org.apache.spark.serializer.{KryoSerializer, SerializerManager} import org.apache.spark.storage.memory.{BlockEvictionHandler, MemoryStore, PartiallySerializedBlock, PartiallyUnrolledIterator} import org.apache.spark.util._ @@ -86,7 +86,7 @@ class MemoryStoreSuite assert(memoryStore.currentUnrollMemoryForThisTask === 0) def reserveUnrollMemoryForThisTask(memory: Long): Boolean = { - memoryStore.reserveUnrollMemoryForThisTask(TestBlockId(""), memory) + memoryStore.reserveUnrollMemoryForThisTask(TestBlockId(""), memory, MemoryMode.ON_HEAP) } // Reserve @@ -99,9 +99,9 @@ class MemoryStoreSuite assert(!reserveUnrollMemoryForThisTask(1000000)) assert(memoryStore.currentUnrollMemoryForThisTask === 800) // not granted // Release - memoryStore.releaseUnrollMemoryForThisTask(100) + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 100) assert(memoryStore.currentUnrollMemoryForThisTask === 700) - memoryStore.releaseUnrollMemoryForThisTask(100) + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 100) assert(memoryStore.currentUnrollMemoryForThisTask === 600) // Reserve again assert(reserveUnrollMemoryForThisTask(4400)) @@ -109,9 +109,9 @@ class MemoryStoreSuite assert(!reserveUnrollMemoryForThisTask(20000)) assert(memoryStore.currentUnrollMemoryForThisTask === 5000) // not granted // Release again - memoryStore.releaseUnrollMemoryForThisTask(1000) + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 1000) assert(memoryStore.currentUnrollMemoryForThisTask === 4000) - memoryStore.releaseUnrollMemoryForThisTask() // release all + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP) // release all assert(memoryStore.currentUnrollMemoryForThisTask === 0) } @@ -254,7 +254,7 @@ class MemoryStoreSuite assert(blockInfoManager.lockNewBlockForWriting( blockId, new BlockInfo(StorageLevel.MEMORY_ONLY_SER, classTag, tellMaster = false))) - val res = memoryStore.putIteratorAsBytes(blockId, iter, classTag) + val res = memoryStore.putIteratorAsBytes(blockId, iter, classTag, MemoryMode.ON_HEAP) blockInfoManager.unlock(blockId) res } @@ -312,7 +312,7 @@ class MemoryStoreSuite assert(blockInfoManager.lockNewBlockForWriting( "b1", new BlockInfo(StorageLevel.MEMORY_ONLY_SER, ClassTag.Any, tellMaster = false))) - val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any) + val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any, MemoryMode.ON_HEAP) blockInfoManager.unlock("b1") assert(res.isLeft) assert(memoryStore.currentUnrollMemoryForThisTask > 0) @@ -333,7 +333,7 @@ class MemoryStoreSuite assert(blockInfoManager.lockNewBlockForWriting( "b1", new BlockInfo(StorageLevel.MEMORY_ONLY_SER, ClassTag.Any, tellMaster = false))) - val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any) + val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any, MemoryMode.ON_HEAP) blockInfoManager.unlock("b1") assert(res.isLeft) assert(memoryStore.currentUnrollMemoryForThisTask > 0) @@ -395,7 +395,7 @@ class MemoryStoreSuite val blockId = BlockId("rdd_3_10") blockInfoManager.lockNewBlockForWriting( blockId, new BlockInfo(StorageLevel.MEMORY_ONLY, ClassTag.Any, tellMaster = false)) - memoryStore.putBytes(blockId, 13000, () => { + memoryStore.putBytes(blockId, 13000, MemoryMode.ON_HEAP, () => { fail("A big ByteBuffer that cannot be put into MemoryStore should not be created") }) } @@ -404,7 +404,7 @@ class MemoryStoreSuite val (memoryStore, _) = makeMemoryStore(12000) val blockId = BlockId("rdd_3_10") var bytes: ChunkedByteBuffer = null - memoryStore.putBytes(blockId, 10000, () => { + memoryStore.putBytes(blockId, 10000, MemoryMode.ON_HEAP, () => { bytes = new ChunkedByteBuffer(ByteBuffer.allocate(10000)) bytes }) diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 14daa003bc..9835f11a2f 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -82,48 +82,51 @@ class StorageStatusListenerSuite extends SparkFunSuite { assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } - test("task end with updated blocks") { + test("updated blocks") { val listener = new StorageStatusListener(conf) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) - val taskMetrics1 = new TaskMetrics - val taskMetrics2 = new TaskMetrics - val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L)) - val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L)) - val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L)) - taskMetrics1.setUpdatedBlockStatuses(Seq(block1, block2)) - taskMetrics2.setUpdatedBlockStatuses(Seq(block3)) - - // Task end with new blocks + + val blockUpdateInfos1 = Seq( + BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0L, 100L), + BlockUpdatedInfo(bm1, RDDBlockId(1, 2), StorageLevel.DISK_ONLY, 0L, 200L) + ) + val blockUpdateInfos2 = + Seq(BlockUpdatedInfo(bm2, RDDBlockId(4, 0), StorageLevel.DISK_ONLY, 0L, 300L)) + + // Add some new blocks assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) + postUpdateBlock(listener, blockUpdateInfos1) assert(listener.executorIdToStorageStatus("big").numBlocks === 2) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2)) + postUpdateBlock(listener, blockUpdateInfos2) assert(listener.executorIdToStorageStatus("big").numBlocks === 2) assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) - // Task end with dropped blocks - val droppedBlock1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.NONE, 0L, 0L)) - val droppedBlock2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.NONE, 0L, 0L)) - val droppedBlock3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.NONE, 0L, 0L)) - taskMetrics1.setUpdatedBlockStatuses(Seq(droppedBlock1, droppedBlock3)) - taskMetrics2.setUpdatedBlockStatuses(Seq(droppedBlock2, droppedBlock3)) + // Dropped the blocks + val droppedBlockInfo1 = Seq( + BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.NONE, 0L, 0L), + BlockUpdatedInfo(bm1, RDDBlockId(4, 0), StorageLevel.NONE, 0L, 0L) + ) + val droppedBlockInfo2 = Seq( + BlockUpdatedInfo(bm2, RDDBlockId(1, 2), StorageLevel.NONE, 0L, 0L), + BlockUpdatedInfo(bm2, RDDBlockId(4, 0), StorageLevel.NONE, 0L, 0L) + ) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) + postUpdateBlock(listener, droppedBlockInfo1) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2)) + postUpdateBlock(listener, droppedBlockInfo2) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) @@ -134,15 +137,14 @@ class StorageStatusListenerSuite extends SparkFunSuite { test("unpersist RDD") { val listener = new StorageStatusListener(conf) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) - val taskMetrics1 = new TaskMetrics - val taskMetrics2 = new TaskMetrics - val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L)) - val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L)) - val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L)) - taskMetrics1.setUpdatedBlockStatuses(Seq(block1, block2)) - taskMetrics2.setUpdatedBlockStatuses(Seq(block3)) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics2)) + val blockUpdateInfos1 = Seq( + BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0L, 100L), + BlockUpdatedInfo(bm1, RDDBlockId(1, 2), StorageLevel.DISK_ONLY, 0L, 200L) + ) + val blockUpdateInfos2 = + Seq(BlockUpdatedInfo(bm1, RDDBlockId(4, 0), StorageLevel.DISK_ONLY, 0L, 300L)) + postUpdateBlock(listener, blockUpdateInfos1) + postUpdateBlock(listener, blockUpdateInfos2) assert(listener.executorIdToStorageStatus("big").numBlocks === 3) // Unpersist RDD @@ -155,4 +157,11 @@ class StorageStatusListenerSuite extends SparkFunSuite { listener.onUnpersistRDD(SparkListenerUnpersistRDD(1)) assert(listener.executorIdToStorageStatus("big").numBlocks === 0) } + + private def postUpdateBlock( + listener: StorageStatusListener, updateBlockInfos: Seq[BlockUpdatedInfo]): Unit = { + updateBlockInfos.foreach { updateBlockInfo => + listener.onBlockUpdated(SparkListenerBlockUpdated(updateBlockInfo)) + } + } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 9876bded33..7d4c0863bc 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -322,11 +322,11 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with assert(stage1Data.inputBytes == 207) assert(stage0Data.outputBytes == 116) assert(stage1Data.outputBytes == 208) - assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get + assert(stage0Data.taskData.get(1234L).get.metrics.get.shuffleReadMetrics.get .totalBlocksFetched == 2) - assert(stage0Data.taskData.get(1235L).get.taskMetrics.get.shuffleReadMetrics.get + assert(stage0Data.taskData.get(1235L).get.metrics.get.shuffleReadMetrics.get .totalBlocksFetched == 102) - assert(stage1Data.taskData.get(1236L).get.taskMetrics.get.shuffleReadMetrics.get + assert(stage1Data.taskData.get(1236L).get.metrics.get.shuffleReadMetrics.get .totalBlocksFetched == 202) // task that was included in a heartbeat @@ -355,9 +355,9 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with assert(stage1Data.inputBytes == 614) assert(stage0Data.outputBytes == 416) assert(stage1Data.outputBytes == 616) - assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get + assert(stage0Data.taskData.get(1234L).get.metrics.get.shuffleReadMetrics.get .totalBlocksFetched == 302) - assert(stage1Data.taskData.get(1237L).get.taskMetrics.get.shuffleReadMetrics.get + assert(stage1Data.taskData.get(1237L).get.metrics.get.shuffleReadMetrics.get .totalBlocksFetched == 402) } } diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index 6b7c538ac8..7d77deeb60 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -106,7 +106,7 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { assert(storageListener.rddInfoList.size === 0) } - test("task end") { + test("block update") { val myRddInfo0 = rddInfo0 val myRddInfo1 = rddInfo1 val myRddInfo2 = rddInfo2 @@ -120,19 +120,13 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { assert(!storageListener._rddInfoMap(1).isCached) assert(!storageListener._rddInfoMap(2).isCached) - // Task end with no updated blocks. This should not change anything. - bus.postToAll(SparkListenerTaskEnd(0, 0, "obliteration", Success, taskInfo, new TaskMetrics)) - assert(storageListener._rddInfoMap.size === 3) - assert(storageListener.rddInfoList.size === 0) - - // Task end with a few new persisted blocks, some from the same RDD - val metrics1 = new TaskMetrics - metrics1.setUpdatedBlockStatuses(Seq( - (RDDBlockId(0, 100), BlockStatus(memAndDisk, 400L, 0L)), - (RDDBlockId(0, 101), BlockStatus(memAndDisk, 0L, 400L)), - (RDDBlockId(1, 20), BlockStatus(memAndDisk, 0L, 240L)) - )) - bus.postToAll(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo, metrics1)) + // Some blocks updated + val blockUpdateInfos = Seq( + BlockUpdatedInfo(bm1, RDDBlockId(0, 100), memAndDisk, 400L, 0L), + BlockUpdatedInfo(bm1, RDDBlockId(0, 101), memAndDisk, 0L, 400L), + BlockUpdatedInfo(bm1, RDDBlockId(1, 20), memAndDisk, 0L, 240L) + ) + postUpdateBlocks(bus, blockUpdateInfos) assert(storageListener._rddInfoMap(0).memSize === 400L) assert(storageListener._rddInfoMap(0).diskSize === 400L) assert(storageListener._rddInfoMap(0).numCachedPartitions === 2) @@ -144,15 +138,14 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { assert(!storageListener._rddInfoMap(2).isCached) assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) - // Task end with a few dropped blocks - val metrics2 = new TaskMetrics - metrics2.setUpdatedBlockStatuses(Seq( - (RDDBlockId(0, 100), BlockStatus(none, 0L, 0L)), - (RDDBlockId(1, 20), BlockStatus(none, 0L, 0L)), - (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L)), // doesn't actually exist - (RDDBlockId(4, 80), BlockStatus(none, 0L, 0L)) // doesn't actually exist - )) - bus.postToAll(SparkListenerTaskEnd(2, 0, "obliteration", Success, taskInfo, metrics2)) + // Drop some blocks + val blockUpdateInfos2 = Seq( + BlockUpdatedInfo(bm1, RDDBlockId(0, 100), none, 0L, 0L), + BlockUpdatedInfo(bm1, RDDBlockId(1, 20), none, 0L, 0L), + BlockUpdatedInfo(bm1, RDDBlockId(2, 40), none, 0L, 0L), // doesn't actually exist + BlockUpdatedInfo(bm1, RDDBlockId(4, 80), none, 0L, 0L) // doesn't actually exist + ) + postUpdateBlocks(bus, blockUpdateInfos2) assert(storageListener._rddInfoMap(0).memSize === 0L) assert(storageListener._rddInfoMap(0).diskSize === 400L) assert(storageListener._rddInfoMap(0).numCachedPartitions === 1) @@ -169,24 +162,27 @@ class StorageTabSuite extends SparkFunSuite with BeforeAndAfter { val rddInfo1 = new RDDInfo(1, "rdd1", 1, memOnly, Seq(4)) val stageInfo0 = new StageInfo(0, 0, "stage0", 1, Seq(rddInfo0), Seq.empty, "details") val stageInfo1 = new StageInfo(1, 0, "stage1", 1, Seq(rddInfo1), Seq.empty, "details") - val taskMetrics0 = new TaskMetrics - val taskMetrics1 = new TaskMetrics - val block0 = (RDDBlockId(0, 1), BlockStatus(memOnly, 100L, 0L)) - val block1 = (RDDBlockId(1, 1), BlockStatus(memOnly, 200L, 0L)) - taskMetrics0.setUpdatedBlockStatuses(Seq(block0)) - taskMetrics1.setUpdatedBlockStatuses(Seq(block1)) + val blockUpdateInfos1 = Seq(BlockUpdatedInfo(bm1, RDDBlockId(0, 1), memOnly, 100L, 0L)) + val blockUpdateInfos2 = Seq(BlockUpdatedInfo(bm1, RDDBlockId(1, 1), memOnly, 200L, 0L)) bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener.rddInfoList.size === 0) - bus.postToAll(SparkListenerTaskEnd(0, 0, "big", Success, taskInfo, taskMetrics0)) + postUpdateBlocks(bus, blockUpdateInfos1) assert(storageListener.rddInfoList.size === 1) bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) assert(storageListener.rddInfoList.size === 1) bus.postToAll(SparkListenerStageCompleted(stageInfo0)) assert(storageListener.rddInfoList.size === 1) - bus.postToAll(SparkListenerTaskEnd(1, 0, "small", Success, taskInfo1, taskMetrics1)) + postUpdateBlocks(bus, blockUpdateInfos2) assert(storageListener.rddInfoList.size === 2) bus.postToAll(SparkListenerStageCompleted(stageInfo1)) assert(storageListener.rddInfoList.size === 2) } + + private def postUpdateBlocks( + bus: SparkListenerBus, blockUpdateInfos: Seq[BlockUpdatedInfo]): Unit = { + blockUpdateInfos.foreach { blockUpdateInfo => + bus.postToAll(SparkListenerBlockUpdated(blockUpdateInfo)) + } + } } diff --git a/core/src/test/scala/org/apache/spark/util/CausedBySuite.scala b/core/src/test/scala/org/apache/spark/util/CausedBySuite.scala new file mode 100644 index 0000000000..4a80e3f1f4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/CausedBySuite.scala @@ -0,0 +1,56 @@ +/* + * 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 org.apache.spark.SparkFunSuite + +class CausedBySuite extends SparkFunSuite { + + test("For an error without a cause, should return the error") { + val error = new Exception + + val causedBy = error match { + case CausedBy(e) => e + } + + assert(causedBy === error) + } + + test("For an error with a cause, should return the cause of the error") { + val cause = new Exception + val error = new Exception(cause) + + val causedBy = error match { + case CausedBy(e) => e + } + + assert(causedBy === cause) + } + + test("For an error with a cause that itself has a cause, return the root cause") { + val causeOfCause = new Exception + val cause = new Exception(causeOfCause) + val error = new Exception(cause) + + val causedBy = error match { + case CausedBy(e) => e + } + + assert(causedBy === causeOfCause) + } +} 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 280e496498..4fa9f9a8f5 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -201,24 +201,29 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { // Make sure only logging errors val logger = Logger.getRootLogger + val oldLogLevel = logger.getLevel logger.setLevel(Level.ERROR) - logger.addAppender(mockAppender) + try { + logger.addAppender(mockAppender) - val testOutputStream = new PipedOutputStream() - val testInputStream = new PipedInputStream(testOutputStream) + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream) - // Close the stream before appender tries to read will cause an IOException - testInputStream.close() - testOutputStream.close() - val appender = FileAppender(testInputStream, testFile, new SparkConf) + // Close the stream before appender tries to read will cause an IOException + testInputStream.close() + testOutputStream.close() + val appender = FileAppender(testInputStream, testFile, new SparkConf) - appender.awaitTermination() + appender.awaitTermination() - // If InputStream was closed without first stopping the appender, an exception will be logged - verify(mockAppender, atLeast(1)).doAppend(loggingEventCaptor.capture) - val loggingEvent = loggingEventCaptor.getValue - assert(loggingEvent.getThrowableInformation !== null) - assert(loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) + // If InputStream was closed without first stopping the appender, an exception will be logged + verify(mockAppender, atLeast(1)).doAppend(loggingEventCaptor.capture) + val loggingEvent = loggingEventCaptor.getValue + assert(loggingEvent.getThrowableInformation !== null) + assert(loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) + } finally { + logger.setLevel(oldLogLevel) + } } test("file appender async close stream gracefully") { @@ -228,30 +233,35 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { // Make sure only logging errors val logger = Logger.getRootLogger + val oldLogLevel = logger.getLevel logger.setLevel(Level.ERROR) - logger.addAppender(mockAppender) + try { + logger.addAppender(mockAppender) - val testOutputStream = new PipedOutputStream() - val testInputStream = new PipedInputStream(testOutputStream) with LatchedInputStream + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream) with LatchedInputStream - // Close the stream before appender tries to read will cause an IOException - testInputStream.close() - testOutputStream.close() - val appender = FileAppender(testInputStream, testFile, new SparkConf) + // Close the stream before appender tries to read will cause an IOException + testInputStream.close() + testOutputStream.close() + val appender = FileAppender(testInputStream, testFile, new SparkConf) - // Stop the appender before an IOException is called during read - testInputStream.latchReadStarted.await() - appender.stop() - testInputStream.latchReadProceed.countDown() + // Stop the appender before an IOException is called during read + testInputStream.latchReadStarted.await() + appender.stop() + testInputStream.latchReadProceed.countDown() - appender.awaitTermination() + appender.awaitTermination() - // Make sure no IOException errors have been logged as a result of appender closing gracefully - verify(mockAppender, atLeast(0)).doAppend(loggingEventCaptor.capture) - import scala.collection.JavaConverters._ - loggingEventCaptor.getAllValues.asScala.foreach { loggingEvent => - assert(loggingEvent.getThrowableInformation === null - || !loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) + // Make sure no IOException errors have been logged as a result of appender closing gracefully + verify(mockAppender, atLeast(0)).doAppend(loggingEventCaptor.capture) + import scala.collection.JavaConverters._ + loggingEventCaptor.getAllValues.asScala.foreach { loggingEvent => + assert(loggingEvent.getThrowableInformation === null + || !loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) + } + } finally { + logger.setLevel(oldLogLevel) } } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 6a2d4c9f2c..de6f408fa8 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -853,7 +853,7 @@ private[spark] object JsonProtocolSuite extends Assertions { if (hasHadoopInput) { val inputMetrics = t.registerInputMetrics(DataReadMethod.Hadoop) inputMetrics.setBytesRead(d + e + f) - inputMetrics.incRecordsReadInternal(if (hasRecords) (d + e + f) / 100 else -1) + inputMetrics.incRecordsRead(if (hasRecords) (d + e + f) / 100 else -1) } else { val sr = t.registerTempShuffleReadMetrics() sr.incRemoteBytesRead(b + d) diff --git a/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala new file mode 100644 index 0000000000..39b31f8dde --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/UninterruptibleThreadSuite.scala @@ -0,0 +1,159 @@ +/* + * 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.util.concurrent.{CountDownLatch, TimeUnit} + +import scala.util.Random + +import com.google.common.util.concurrent.Uninterruptibles + +import org.apache.spark.SparkFunSuite + +class UninterruptibleThreadSuite extends SparkFunSuite { + + /** Sleep millis and return true if it's interrupted */ + private def sleep(millis: Long): Boolean = { + try { + Thread.sleep(millis) + false + } catch { + case _: InterruptedException => + true + } + } + + test("interrupt when runUninterruptibly is running") { + val enterRunUninterruptibly = new CountDownLatch(1) + @volatile var hasInterruptedException = false + @volatile var interruptStatusBeforeExit = false + val t = new UninterruptibleThread("test") { + override def run(): Unit = { + runUninterruptibly { + enterRunUninterruptibly.countDown() + hasInterruptedException = sleep(1000) + } + interruptStatusBeforeExit = Thread.interrupted() + } + } + t.start() + assert(enterRunUninterruptibly.await(10, TimeUnit.SECONDS), "await timeout") + t.interrupt() + t.join() + assert(hasInterruptedException === false) + assert(interruptStatusBeforeExit === true) + } + + test("interrupt before runUninterruptibly runs") { + val interruptLatch = new CountDownLatch(1) + @volatile var hasInterruptedException = false + @volatile var interruptStatusBeforeExit = false + val t = new UninterruptibleThread("test") { + override def run(): Unit = { + Uninterruptibles.awaitUninterruptibly(interruptLatch, 10, TimeUnit.SECONDS) + try { + runUninterruptibly { + assert(false, "Should not reach here") + } + } catch { + case _: InterruptedException => hasInterruptedException = true + } + interruptStatusBeforeExit = Thread.interrupted() + } + } + t.start() + t.interrupt() + interruptLatch.countDown() + t.join() + assert(hasInterruptedException === true) + assert(interruptStatusBeforeExit === false) + } + + test("nested runUninterruptibly") { + val enterRunUninterruptibly = new CountDownLatch(1) + val interruptLatch = new CountDownLatch(1) + @volatile var hasInterruptedException = false + @volatile var interruptStatusBeforeExit = false + val t = new UninterruptibleThread("test") { + override def run(): Unit = { + runUninterruptibly { + enterRunUninterruptibly.countDown() + Uninterruptibles.awaitUninterruptibly(interruptLatch, 10, TimeUnit.SECONDS) + hasInterruptedException = sleep(1) + runUninterruptibly { + if (sleep(1)) { + hasInterruptedException = true + } + } + if (sleep(1)) { + hasInterruptedException = true + } + } + interruptStatusBeforeExit = Thread.interrupted() + } + } + t.start() + assert(enterRunUninterruptibly.await(10, TimeUnit.SECONDS), "await timeout") + t.interrupt() + interruptLatch.countDown() + t.join() + assert(hasInterruptedException === false) + assert(interruptStatusBeforeExit === true) + } + + test("stress test") { + @volatile var hasInterruptedException = false + val t = new UninterruptibleThread("test") { + override def run(): Unit = { + for (i <- 0 until 100) { + try { + runUninterruptibly { + if (sleep(Random.nextInt(10))) { + hasInterruptedException = true + } + runUninterruptibly { + if (sleep(Random.nextInt(10))) { + hasInterruptedException = true + } + } + if (sleep(Random.nextInt(10))) { + hasInterruptedException = true + } + } + Uninterruptibles.sleepUninterruptibly(Random.nextInt(10), TimeUnit.MILLISECONDS) + // 50% chance to clear the interrupted status + if (Random.nextBoolean()) { + Thread.interrupted() + } + } catch { + case _: InterruptedException => + // The first runUninterruptibly may throw InterruptedException if the interrupt status + // is set before running `f`. + } + } + } + } + t.start() + for (i <- 0 until 400) { + Thread.sleep(Random.nextInt(10)) + t.interrupt() + } + t.join() + assert(hasInterruptedException === false) + } +} diff --git a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala index 361ec95654..226622075a 100644 --- a/core/src/test/scala/org/apache/spark/util/io/ByteArrayChunkOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala @@ -17,48 +17,53 @@ package org.apache.spark.util.io +import java.nio.ByteBuffer + import scala.util.Random import org.apache.spark.SparkFunSuite -class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { +class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { test("empty output") { - val o = new ByteArrayChunkOutputStream(1024) - assert(o.toArrays.length === 0) + val o = new ChunkedByteBufferOutputStream(1024, ByteBuffer.allocate) + assert(o.toChunkedByteBuffer.size === 0) } test("write a single byte") { - val o = new ByteArrayChunkOutputStream(1024) + val o = new ChunkedByteBufferOutputStream(1024, ByteBuffer.allocate) o.write(10) - assert(o.toArrays.length === 1) - assert(o.toArrays.head.toSeq === Seq(10.toByte)) + val chunkedByteBuffer = o.toChunkedByteBuffer + assert(chunkedByteBuffer.getChunks().length === 1) + assert(chunkedByteBuffer.getChunks().head.array().toSeq === Seq(10.toByte)) } test("write a single near boundary") { - val o = new ByteArrayChunkOutputStream(10) + val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(new Array[Byte](9)) o.write(99) - assert(o.toArrays.length === 1) - assert(o.toArrays.head(9) === 99.toByte) + val chunkedByteBuffer = o.toChunkedByteBuffer + assert(chunkedByteBuffer.getChunks().length === 1) + assert(chunkedByteBuffer.getChunks().head.array()(9) === 99.toByte) } test("write a single at boundary") { - val o = new ByteArrayChunkOutputStream(10) + val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(new Array[Byte](10)) o.write(99) - assert(o.toArrays.length === 2) - assert(o.toArrays(1).length === 1) - assert(o.toArrays(1)(0) === 99.toByte) + val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) + assert(arrays.length === 2) + assert(arrays(1).length === 1) + assert(arrays(1)(0) === 99.toByte) } test("single chunk output") { val ref = new Array[Byte](8) Random.nextBytes(ref) - val o = new ByteArrayChunkOutputStream(10) + val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) - val arrays = o.toArrays + val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 1) assert(arrays.head.length === ref.length) assert(arrays.head.toSeq === ref.toSeq) @@ -67,9 +72,9 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { test("single chunk output at boundary size") { val ref = new Array[Byte](10) Random.nextBytes(ref) - val o = new ByteArrayChunkOutputStream(10) + val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) - val arrays = o.toArrays + val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 1) assert(arrays.head.length === ref.length) assert(arrays.head.toSeq === ref.toSeq) @@ -78,9 +83,9 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { test("multiple chunk output") { val ref = new Array[Byte](26) Random.nextBytes(ref) - val o = new ByteArrayChunkOutputStream(10) + val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) - val arrays = o.toArrays + val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 3) assert(arrays(0).length === 10) assert(arrays(1).length === 10) @@ -94,9 +99,9 @@ class ByteArrayChunkOutputStreamSuite extends SparkFunSuite { test("multiple chunk output at boundary size") { val ref = new Array[Byte](30) Random.nextBytes(ref) - val o = new ByteArrayChunkOutputStream(10) + val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) - val arrays = o.toArrays + val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 3) assert(arrays(0).length === 10) assert(arrays(1).length === 10) diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index 791491daf0..7eb2f56c20 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -129,6 +129,13 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers { t(m / 2) } + def replacementSampling(data: Iterator[Int], sampler: PoissonSampler[Int]): Iterator[Int] = { + data.flatMap { item => + val count = sampler.sample() + if (count == 0) Iterator.empty else Iterator.fill(count)(item) + } + } + test("utilities") { val s1 = Array(0, 1, 1, 0, 2) val s2 = Array(1, 0, 3, 2, 1) @@ -189,6 +196,36 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers { d should be > D } + test("bernoulli sampling without iterator") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + val data = Iterator.from(0) + + var sampler: RandomSampler[Int, Int] = new BernoulliSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.5))) + d should be < D + + sampler = new BernoulliSampler[Int](0.7) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.7))) + d should be < D + + sampler = new BernoulliSampler[Int](0.9) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.9))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new BernoulliSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.6))) + d should be > D + } + test("bernoulli sampling with gap sampling optimization") { // Tests expect maximum gap sampling fraction to be this value RandomSampler.defaultMaxGapSamplingFraction should be (0.4) @@ -217,6 +254,37 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers { d should be > D } + test("bernoulli sampling (without iterator) with gap sampling optimization") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + val data = Iterator.from(0) + + var sampler: RandomSampler[Int, Int] = new BernoulliSampler[Int](0.01) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), + gaps(sample(Iterator.from(0), 0.01))) + d should be < D + + sampler = new BernoulliSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + sampler = new BernoulliSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.3))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new BernoulliSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.4))) + d should be > D + } + test("bernoulli boundary cases") { val data = (1 to 100).toArray @@ -233,6 +301,22 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers { sampler.sample(data.iterator).toArray should be (data) } + test("bernoulli (without iterator) boundary cases") { + val data = (1 to 100).toArray + + var sampler = new BernoulliSampler[Int](0.0) + data.filter(_ => sampler.sample() > 0) should be (Array.empty[Int]) + + sampler = new BernoulliSampler[Int](1.0) + data.filter(_ => sampler.sample() > 0) should be (data) + + sampler = new BernoulliSampler[Int](0.0 - (RandomSampler.roundingEpsilon / 2.0)) + data.filter(_ => sampler.sample() > 0) should be (Array.empty[Int]) + + sampler = new BernoulliSampler[Int](1.0 + (RandomSampler.roundingEpsilon / 2.0)) + data.filter(_ => sampler.sample() > 0) should be (data) + } + test("bernoulli data types") { // Tests expect maximum gap sampling fraction to be this value RandomSampler.defaultMaxGapSamplingFraction should be (0.4) @@ -341,6 +425,36 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers { d should be > D } + test("replacement sampling without iterator") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + val data = Iterator.from(0) + + var sampler = new PoissonSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.5))) + d should be < D + + sampler = new PoissonSampler[Int](0.7) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.7))) + d should be < D + + sampler = new PoissonSampler[Int](0.9) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.9))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new PoissonSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.6))) + d should be > D + } + test("replacement sampling with gap sampling") { // Tests expect maximum gap sampling fraction to be this value RandomSampler.defaultMaxGapSamplingFraction should be (0.4) @@ -369,6 +483,36 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers { d should be > D } + test("replacement sampling (without iterator) with gap sampling") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + val data = Iterator.from(0) + + var sampler = new PoissonSampler[Int](0.01) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.01))) + d should be < D + + sampler = new PoissonSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + + sampler = new PoissonSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.3))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new PoissonSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(replacementSampling(data, sampler)), gaps(sampleWR(Iterator.from(0), 0.4))) + d should be > D + } + test("replacement boundary cases") { val data = (1 to 100).toArray @@ -383,6 +527,20 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers { sampler.sample(data.iterator).length should be > (data.length) } + test("replacement (without) boundary cases") { + val data = (1 to 100).toArray + + var sampler = new PoissonSampler[Int](0.0) + replacementSampling(data.iterator, sampler).toArray should be (Array.empty[Int]) + + sampler = new PoissonSampler[Int](0.0 - (RandomSampler.roundingEpsilon / 2.0)) + replacementSampling(data.iterator, sampler).toArray should be (Array.empty[Int]) + + // sampling with replacement has no upper bound on sampling fraction + sampler = new PoissonSampler[Int](2.0) + replacementSampling(data.iterator, sampler).length should be > (data.length) + } + test("replacement data types") { // Tests expect maximum gap sampling fraction to be this value RandomSampler.defaultMaxGapSamplingFraction should be (0.4) @@ -477,6 +635,22 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers { d should be < D } + test("bernoulli partitioning sampling without iterator") { + var d: Double = 0.0 + + val data = Iterator.from(0) + + var sampler = new BernoulliCellSampler[Int](0.1, 0.2) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + sampler = new BernoulliCellSampler[Int](0.1, 0.2, true) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(data.filter(_ => sampler.sample() > 0)), gaps(sample(Iterator.from(0), 0.9))) + d should be < D + } + test("bernoulli partitioning boundary cases") { val data = (1 to 100).toArray val d = RandomSampler.roundingEpsilon / 2.0 @@ -500,6 +674,29 @@ class RandomSamplerSuite extends SparkFunSuite with Matchers { sampler.sample(data.iterator).toArray should be (Array.empty[Int]) } + test("bernoulli partitioning (without iterator) boundary cases") { + val data = (1 to 100).toArray + val d = RandomSampler.roundingEpsilon / 2.0 + + var sampler = new BernoulliCellSampler[Int](0.0, 0.0) + data.filter(_ => sampler.sample() > 0).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5) + data.filter(_ => sampler.sample() > 0).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](1.0, 1.0) + data.filter(_ => sampler.sample() > 0).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](0.0, 1.0) + data.filter(_ => sampler.sample() > 0).toArray should be (data) + + sampler = new BernoulliCellSampler[Int](0.0 - d, 1.0 + d) + data.filter(_ => sampler.sample() > 0).toArray should be (data) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5 - d) + data.filter(_ => sampler.sample() > 0).toArray should be (Array.empty[Int]) + } + test("bernoulli partitioning data") { val seed = rngSeed.nextLong val data = (1 to 100).toArray |