diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index a1805f57b1dc..88f228621952 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1694,7 +1694,7 @@ test_that("column functions", { # check for unparseable df <- as.DataFrame(list(list("a" = ""))) - expect_equal(collect(select(df, from_json(df$a, schema)))[[1]][[1]]$a, NA) + expect_equal(collect(select(df, from_json(df$a, schema)))[[1]][[1]], NA) # check if array type in string is correctly supported. jsonArr <- "[{\"name\":\"Bob\"}, {\"name\":\"Alice\"}]" diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java index f08d8b0f984c..43c3d23b6304 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/ChunkFetchRequestHandler.java @@ -90,7 +90,6 @@ protected void channelRead0( ManagedBuffer buf; try { streamManager.checkAuthorization(client, msg.streamChunkId.streamId); - streamManager.registerChannel(channel, msg.streamChunkId.streamId); buf = streamManager.getChunk(msg.streamChunkId.streamId, msg.streamChunkId.chunkIndex); } catch (Exception e) { logger.error(String.format("Error opening block %s for request from %s", diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index 0f6a8824d95e..6fafcc131fa2 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import io.netty.channel.Channel; import org.apache.commons.lang3.tuple.ImmutablePair; @@ -49,7 +50,7 @@ private static class StreamState { final Iterator buffers; // The channel associated to the stream - Channel associatedChannel = null; + final Channel associatedChannel; // Used to keep track of the index of the buffer that the user has retrieved, just to ensure // that the caller only requests each chunk one at a time, in order. @@ -58,9 +59,10 @@ private static class StreamState { // Used to keep track of the number of chunks being transferred and not finished yet. volatile long chunksBeingTransferred = 0L; - StreamState(String appId, Iterator buffers) { + StreamState(String appId, Iterator buffers, Channel channel) { this.appId = appId; this.buffers = Preconditions.checkNotNull(buffers); + this.associatedChannel = channel; } } @@ -71,13 +73,6 @@ public OneForOneStreamManager() { streams = new ConcurrentHashMap<>(); } - @Override - public void registerChannel(Channel channel, long streamId) { - if (streams.containsKey(streamId)) { - streams.get(streamId).associatedChannel = channel; - } - } - @Override public ManagedBuffer getChunk(long streamId, int chunkIndex) { StreamState state = streams.get(streamId); @@ -195,11 +190,19 @@ public long chunksBeingTransferred() { * * If an app ID is provided, only callers who've authenticated with the given app ID will be * allowed to fetch from this stream. + * + * This method also associates the stream with a single client connection, which is guaranteed + * to be the only reader of the stream. Once the connection is closed, the stream will never + * be used again, enabling cleanup by `connectionTerminated`. */ - public long registerStream(String appId, Iterator buffers) { + public long registerStream(String appId, Iterator buffers, Channel channel) { long myStreamId = nextStreamId.getAndIncrement(); - streams.put(myStreamId, new StreamState(appId, buffers)); + streams.put(myStreamId, new StreamState(appId, buffers, channel)); return myStreamId; } + @VisibleForTesting + public int numStreamStates() { + return streams.size(); + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java index c53529583160..e48d27be1126 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java @@ -60,16 +60,6 @@ public ManagedBuffer openStream(String streamId) { throw new UnsupportedOperationException(); } - /** - * Associates a stream with a single client connection, which is guaranteed to be the only reader - * of the stream. The getChunk() method will be called serially on this connection and once the - * connection is closed, the stream will never be used again, enabling cleanup. - * - * This must be called before the first getChunk() on the stream, but it may be invoked multiple - * times with the same channel and stream id. - */ - public void registerChannel(Channel channel, long streamId) { } - /** * Indicates that the given channel has been terminated. After this occurs, we are guaranteed not * to read from the associated streams again, so any state can be cleaned up. diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java index a0ecde292646..9b327d5404b4 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -126,6 +126,10 @@ private void init(String hostToBind, int portToBind) { bootstrap.childOption(ChannelOption.SO_SNDBUF, conf.sendBuf()); } + if (conf.enableTcpKeepAlive()) { + bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true); + } + bootstrap.childHandler(new ChannelInitializer() { @Override protected void initChannel(SocketChannel ch) { diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 89ee5ee3c0cd..3628da68f1c6 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -42,6 +42,7 @@ public class TransportConf { private final String SPARK_NETWORK_IO_RETRYWAIT_KEY; private final String SPARK_NETWORK_IO_LAZYFD_KEY; private final String SPARK_NETWORK_VERBOSE_METRICS; + private final String SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY; private final ConfigProvider conf; @@ -64,6 +65,7 @@ public TransportConf(String module, ConfigProvider conf) { SPARK_NETWORK_IO_RETRYWAIT_KEY = getConfKey("io.retryWait"); SPARK_NETWORK_IO_LAZYFD_KEY = getConfKey("io.lazyFD"); SPARK_NETWORK_VERBOSE_METRICS = getConfKey("io.enableVerboseMetrics"); + SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY = getConfKey("io.enableTcpKeepAlive"); } public int getInt(String name, int defaultValue) { @@ -173,6 +175,14 @@ public boolean verboseMetrics() { return conf.getBoolean(SPARK_NETWORK_VERBOSE_METRICS, false); } + /** + * Whether to enable TCP keep-alive. If true, the TCP keep-alives are enabled, which removes + * connections that are idle for too long. + */ + public boolean enableTcpKeepAlive() { + return conf.getBoolean(SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY, false); + } + /** * Maximum number of retries when binding to a port before giving up. */ diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index 37a8664a5266..ab4dd04a1298 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -172,7 +172,7 @@ public void onFailure(int chunkIndex, Throwable e) { for (int chunkIndex : chunkIndices) { client.fetchChunk(STREAM_ID, chunkIndex, callback); } - if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { + if (!sem.tryAcquire(chunkIndices.size(), 60, TimeUnit.SECONDS)) { fail("Timeout getting response from the server"); } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java index 2c72c53a33ae..6c9239606bb8 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchRequestHandlerSuite.java @@ -64,8 +64,7 @@ public void handleChunkFetchRequest() throws Exception { managedBuffers.add(new TestManagedBuffer(20)); managedBuffers.add(new TestManagedBuffer(30)); managedBuffers.add(new TestManagedBuffer(40)); - long streamId = streamManager.registerStream("test-app", managedBuffers.iterator()); - streamManager.registerChannel(channel, streamId); + long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); TransportClient reverseClient = mock(TransportClient.class); ChunkFetchRequestHandler requestHandler = new ChunkFetchRequestHandler(reverseClient, rpcHandler.getStreamManager(), 2L); diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java index ad640415a8e6..a87f6c11a2bf 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java @@ -58,8 +58,10 @@ public void handleStreamRequest() throws Exception { managedBuffers.add(new TestManagedBuffer(20)); managedBuffers.add(new TestManagedBuffer(30)); managedBuffers.add(new TestManagedBuffer(40)); - long streamId = streamManager.registerStream("test-app", managedBuffers.iterator()); - streamManager.registerChannel(channel, streamId); + long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); + + assert streamManager.numStreamStates() == 1; + TransportClient reverseClient = mock(TransportClient.class); TransportRequestHandler requestHandler = new TransportRequestHandler(channel, reverseClient, rpcHandler, 2L); @@ -94,5 +96,8 @@ public void handleStreamRequest() throws Exception { requestHandler.handle(request3); verify(channel, times(1)).close(); assert responseAndPromisePairs.size() == 3; + + streamManager.connectionTerminated(channel); + assert streamManager.numStreamStates() == 0; } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java index c647525d8f1b..4248762c3238 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java @@ -37,14 +37,15 @@ public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception { TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); buffers.add(buffer1); buffers.add(buffer2); - long streamId = manager.registerStream("appId", buffers.iterator()); Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); - manager.registerChannel(dummyChannel, streamId); + manager.registerStream("appId", buffers.iterator(), dummyChannel); + assert manager.numStreamStates() == 1; manager.connectionTerminated(dummyChannel); Mockito.verify(buffer1, Mockito.times(1)).release(); Mockito.verify(buffer2, Mockito.times(1)).release(); + assert manager.numStreamStates() == 0; } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 788a845c5775..b25e48a164e6 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -92,7 +92,7 @@ protected void handleMessage( OpenBlocks msg = (OpenBlocks) msgObj; checkAuth(client, msg.appId); long streamId = streamManager.registerStream(client.getClientId(), - new ManagedBufferIterator(msg.appId, msg.execId, msg.blockIds)); + new ManagedBufferIterator(msg.appId, msg.execId, msg.blockIds), client.getChannel()); if (logger.isTraceEnabled()) { logger.trace("Registered streamId {} with {} buffers for client {} from host {}", streamId, diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 4cc9a16e1449..537c277cd26b 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -103,7 +103,8 @@ public void testOpenShuffleBlocks() { @SuppressWarnings("unchecked") ArgumentCaptor> stream = (ArgumentCaptor>) (ArgumentCaptor) ArgumentCaptor.forClass(Iterator.class); - verify(streamManager, times(1)).registerStream(anyString(), stream.capture()); + verify(streamManager, times(1)).registerStream(anyString(), stream.capture(), + any()); Iterator buffers = stream.getValue(); assertEquals(block0Marker, buffers.next()); assertEquals(block1Marker, buffers.next()); diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index 2cd39bd60c2a..305cc1c5d111 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -23,7 +23,7 @@ /** * An array of long values. Compared with native JVM arrays, this: *
    - *
  • supports using both in-heap and off-heap memory
  • + *
  • supports using both on-heap and off-heap memory
  • *
  • has no bound checking, and thus can crash the JVM process when assert is turned off
  • *
*/ diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java index 74ebc87dc978..897b8a2b7ec5 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java @@ -21,7 +21,7 @@ /** * A memory location. Tracked either by a memory address (with off-heap allocation), - * or by an offset from a JVM object (in-heap allocation). + * or by an offset from a JVM object (on-heap allocation). */ public class MemoryLocation { diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 28b646ba3c95..1d9391845be5 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -85,9 +85,9 @@ public class TaskMemoryManager { /** * Similar to an operating system's page table, this array maps page numbers into base object * pointers, allowing us to translate between the hashtable's internal 64-bit address - * representation and the baseObject+offset representation which we use to support both in- and + * representation and the baseObject+offset representation which we use to support both on- and * off-heap addresses. When using an off-heap allocator, every entry in this map will be `null`. - * When using an in-heap allocator, the entries in this map will point to pages' base objects. + * When using an on-heap allocator, the entries in this map will point to pages' base objects. * Entries are added to this map as new data pages are allocated. */ private final MemoryBlock[] pageTable = new MemoryBlock[PAGE_TABLE_SIZE]; @@ -102,7 +102,7 @@ public class TaskMemoryManager { private final long taskAttemptId; /** - * Tracks whether we're in-heap or off-heap. For off-heap, we short-circuit most of these methods + * Tracks whether we're on-heap or off-heap. For off-heap, we short-circuit most of these methods * without doing any masking or lookups. Since this branching should be well-predicted by the JIT, * this extra layer of indirection / abstraction hopefully shouldn't be too expensive. */ diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 997bc9e3f043..32b446785a9f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -34,6 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.internal.config.package$; import org.apache.spark.Partitioner; import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; @@ -104,7 +105,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { SparkConf conf, ShuffleWriteMetricsReporter writeMetrics) { // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - this.fileBufferSize = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; + this.fileBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); 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 dc43215373e1..024756087bf7 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 @@ -129,7 +129,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { (int) conf.get(package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD()); this.writeMetrics = writeMetrics; this.inMemSorter = new ShuffleInMemorySorter( - this, initialSize, conf.getBoolean("spark.shuffle.sort.useRadixSort", true)); + this, initialSize, (boolean) conf.get(package$.MODULE$.SHUFFLE_SORT_USE_RADIXSORT())); this.peakMemoryUsedBytes = getMemoryUsage(); this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 4b0c74341551..36081069b0e7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -37,6 +37,7 @@ import org.apache.spark.*; import org.apache.spark.annotation.Private; +import org.apache.spark.internal.config.package$; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.NioBufferedFileInputStream; @@ -55,7 +56,6 @@ import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; -import org.apache.spark.internal.config.package$; @Private public class UnsafeShuffleWriter extends ShuffleWriter { @@ -143,8 +143,8 @@ public UnsafeShuffleWriter( this.taskContext = taskContext; this.sparkConf = sparkConf; this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); - this.initialSortBufferSize = sparkConf.getInt("spark.shuffle.sort.initialBufferSize", - DEFAULT_INITIAL_SORT_BUFFER_SIZE); + this.initialSortBufferSize = + (int) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()); this.inputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.outputBufferSizeInBytes = @@ -282,10 +282,10 @@ void forceSorterToSpill() throws IOException { * @return the partition lengths in the merged file. */ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { - final boolean compressionEnabled = sparkConf.getBoolean("spark.shuffle.compress", true); + final boolean compressionEnabled = (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_COMPRESS()); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = - sparkConf.getBoolean("spark.shuffle.unsafe.fastMergeEnabled", true); + (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_UNDAFE_FAST_MERGE_ENABLE()); final boolean fastMergeIsSupported = !compressionEnabled || CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); 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 2ff98a69ee1f..13ca7fb8a318 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 @@ -854,7 +854,7 @@ public long getPeakMemoryUsedBytes() { /** * Returns the average number of probes per key lookup. */ - public double getAverageProbesPerLookup() { + public double getAvgHashProbeBucketListIterations() { return (1.0 * numProbes) / numKeyLookups; } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java index fb179d07edeb..99303e60e4a3 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java @@ -21,13 +21,13 @@ import com.google.common.io.Closeables; import org.apache.spark.SparkEnv; import org.apache.spark.TaskContext; +import org.apache.spark.internal.config.package$; +import org.apache.spark.internal.config.ConfigEntry; import org.apache.spark.io.NioBufferedFileInputStream; import org.apache.spark.io.ReadAheadInputStream; import org.apache.spark.serializer.SerializerManager; import org.apache.spark.storage.BlockId; import org.apache.spark.unsafe.Platform; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.*; @@ -36,9 +36,7 @@ * of the file format). */ public final class UnsafeSorterSpillReader extends UnsafeSorterIterator implements Closeable { - private static final Logger logger = LoggerFactory.getLogger(UnsafeSorterSpillReader.class); - private static final int DEFAULT_BUFFER_SIZE_BYTES = 1024 * 1024; // 1 MB - private static final int MAX_BUFFER_SIZE_BYTES = 16777216; // 16 mb + public static final int MAX_BUFFER_SIZE_BYTES = 16777216; // 16 mb private InputStream in; private DataInputStream din; @@ -59,28 +57,23 @@ public UnsafeSorterSpillReader( File file, BlockId blockId) throws IOException { assert (file.length() > 0); - long bufferSizeBytes = - SparkEnv.get() == null ? - DEFAULT_BUFFER_SIZE_BYTES: - SparkEnv.get().conf().getSizeAsBytes("spark.unsafe.sorter.spill.reader.buffer.size", - DEFAULT_BUFFER_SIZE_BYTES); - if (bufferSizeBytes > MAX_BUFFER_SIZE_BYTES || bufferSizeBytes < DEFAULT_BUFFER_SIZE_BYTES) { - // fall back to a sane default value - logger.warn("Value of config \"spark.unsafe.sorter.spill.reader.buffer.size\" = {} not in " + - "allowed range [{}, {}). Falling back to default value : {} bytes", bufferSizeBytes, - DEFAULT_BUFFER_SIZE_BYTES, MAX_BUFFER_SIZE_BYTES, DEFAULT_BUFFER_SIZE_BYTES); - bufferSizeBytes = DEFAULT_BUFFER_SIZE_BYTES; - } + final ConfigEntry bufferSizeConfigEntry = + package$.MODULE$.UNSAFE_SORTER_SPILL_READER_BUFFER_SIZE(); + // This value must be less than or equal to MAX_BUFFER_SIZE_BYTES. Cast to int is always safe. + final int DEFAULT_BUFFER_SIZE_BYTES = + ((Long) bufferSizeConfigEntry.defaultValue().get()).intValue(); + int bufferSizeBytes = SparkEnv.get() == null ? DEFAULT_BUFFER_SIZE_BYTES : + ((Long) SparkEnv.get().conf().get(bufferSizeConfigEntry)).intValue(); - final boolean readAheadEnabled = SparkEnv.get() != null && - SparkEnv.get().conf().getBoolean("spark.unsafe.sorter.spill.read.ahead.enabled", true); + final boolean readAheadEnabled = SparkEnv.get() != null && (boolean)SparkEnv.get().conf().get( + package$.MODULE$.UNSAFE_SORTER_SPILL_READ_AHEAD_ENABLED()); final InputStream bs = - new NioBufferedFileInputStream(file, (int) bufferSizeBytes); + new NioBufferedFileInputStream(file, bufferSizeBytes); try { if (readAheadEnabled) { this.in = new ReadAheadInputStream(serializerManager.wrapStream(blockId, bs), - (int) bufferSizeBytes); + bufferSizeBytes); } else { this.in = serializerManager.wrapStream(blockId, bs); } diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index a48c02ae279b..98d67c91f24b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -114,7 +114,6 @@ $(document).ready(function () { var endPoint = createRESTEndPointForExecutorsPage(appId); $.getJSON(endPoint, function (response, status, jqXHR) { - var summary = []; var allExecCnt = 0; var allRDDBlocks = 0; var allMemoryUsed = 0; @@ -505,7 +504,7 @@ $(document).ready(function () { {data: 'allTotalTasks'}, { data: function (row, type) { - return type === 'display' ? (formatDuration(row.allTotalDuration, type) + ' (' + formatDuration(row.allTotalGCTime, type) + ')') : row.allTotalDuration + return type === 'display' ? (formatDuration(row.allTotalDuration) + ' (' + formatDuration(row.allTotalGCTime) + ')') : row.allTotalDuration }, "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { if (oData.allTotalDuration > 0) { diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index abc2ec0fa653..35e4de9ac664 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -103,12 +103,12 @@ $(document).ready(function() { pageLength: 20 }); - historySummary = $("#history-summary"); - searchString = historySummary["context"]["location"]["search"]; - requestedIncomplete = getParameterByName("showIncomplete", searchString); + var historySummary = $("#history-summary"); + var searchString = historySummary["context"]["location"]["search"]; + var requestedIncomplete = getParameterByName("showIncomplete", searchString); requestedIncomplete = (requestedIncomplete == "true" ? true : false); - appParams = { + var appParams = { limit: appLimit, status: (requestedIncomplete ? "running" : "completed") }; @@ -116,7 +116,7 @@ $(document).ready(function() { $.getJSON(uiRoot + "/api/v1/applications", appParams, function(response,status,jqXHR) { var array = []; var hasMultipleAttempts = false; - for (i in response) { + for (var i in response) { var app = response[i]; if (app["attempts"][0]["completed"] == requestedIncomplete) { continue; // if we want to show for Incomplete, we skip the completed apps; otherwise skip incomplete ones. @@ -127,7 +127,7 @@ $(document).ready(function() { hasMultipleAttempts = true; } var num = app["attempts"].length; - for (j in app["attempts"]) { + for (var j in app["attempts"]) { var attempt = app["attempts"][j]; attempt["startTime"] = formatTimeMillis(attempt["startTimeEpoch"]); attempt["endTime"] = formatTimeMillis(attempt["endTimeEpoch"]); @@ -149,7 +149,7 @@ $(document).ready(function() { "applications": array, "hasMultipleAttempts": hasMultipleAttempts, "showCompletedColumns": !requestedIncomplete, - } + }; $.get(uiRoot + "/static/historypage-template.html", function(template) { var sibling = historySummary.prev(); @@ -157,7 +157,7 @@ $(document).ready(function() { var apps = $(Mustache.render($(template).filter("#history-summary-template").html(),data)); var attemptIdColumnName = 'attemptId'; var startedColumnName = 'started'; - var defaultSortColumn = completedColumnName = 'completed'; + var completedColumnName = 'completed'; var durationColumnName = 'duration'; var conf = { "columns": [ diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js index 75b959fdeb59..cf508ac573f3 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js @@ -220,7 +220,7 @@ function renderDagVizForJob(svgContainer) { } else { // Link each graph to the corresponding stage page (TODO: handle stage attempts) // Use the link from the stage table so it also works for the history server - var attemptId = 0 + var attemptId = 0; var stageLink = d3.select("#stage-" + stageId + "-" + attemptId) .select("a.name-link") .attr("href"); @@ -236,7 +236,7 @@ function renderDagVizForJob(svgContainer) { // existing ones, taking into account the position and width of the last stage's // container. We do not need to do this for the first stage of this job. if (i > 0) { - var existingStages = svgContainer.selectAll("g.cluster.stage") + var existingStages = svgContainer.selectAll("g.cluster.stage"); if (!existingStages.empty()) { var lastStage = d3.select(existingStages[0].pop()); var lastStageWidth = toFloat(lastStage.select("rect").attr("width")); @@ -369,8 +369,8 @@ function resizeSvg(svg) { * here this function is to enable line break. */ function interpretLineBreak(svg) { - var allTSpan = svg.selectAll("tspan").each(function() { - node = d3.select(this); + svg.selectAll("tspan").each(function() { + var node = d3.select(this); var original = node[0][0].innerHTML; if (original.indexOf("\\n") != -1) { var arr = original.split("\\n"); diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 5b792ffc584d..4fe2cd95c673 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -263,7 +263,6 @@ function reselectCheckboxesBasedOnTaskTableState() { function getStageAttemptId() { var words = document.baseURI.split('?'); - var attemptIdStr = words[1].split('&')[1]; var digitsRegex = /[0-9]+/; // We are using regex here to extract the stage attempt id as there might be certain url's with format // like /proxy/application_1539986433979_27115/stages/stage/?id=0&attempt=0#tasksTitle @@ -433,7 +432,7 @@ $(document).ready(function () { "oLanguage": { "sEmptyTable": "No data to show yet" } - } + }; var executorSummaryTableSelector = $("#summary-executor-table").DataTable(executorSummaryConf); $('#parent-container [data-toggle="tooltip"]').tooltip(); @@ -612,7 +611,7 @@ $(document).ready(function () { "searching": false, "order": [[0, "asc"]], "bAutoWidth": false - } + }; $("#accumulator-table").DataTable(accumulatorConf); // building tasks table that uses server side functionality diff --git a/core/src/main/resources/org/apache/spark/ui/static/table.js b/core/src/main/resources/org/apache/spark/ui/static/table.js index 0315ebf5c48a..fd258d5ee70d 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/table.js +++ b/core/src/main/resources/org/apache/spark/ui/static/table.js @@ -89,7 +89,7 @@ function onSearchStringChange() { if($(this).attr('id') && $(this).attr('id').match(/thread_[0-9]+_tr/) ) { var children = $(this).children() var found = false - for (i = 0; i < children.length; i++) { + for (var i = 0; i < children.length; i++) { if (children.eq(i).text().toLowerCase().indexOf(searchString) >= 0) { found = true } diff --git a/core/src/main/resources/org/apache/spark/ui/static/utils.js b/core/src/main/resources/org/apache/spark/ui/static/utils.js index 22985e31a780..6fc34a9e1f7e 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/utils.js +++ b/core/src/main/resources/org/apache/spark/ui/static/utils.js @@ -170,7 +170,7 @@ function createRESTEndPointForExecutorsPage(appId) { if (ind > 0) { var appId = words[ind + 1]; var newBaseURI = words.slice(0, ind + 2).join('/'); - return newBaseURI + "/api/v1/applications/" + appId + "/allexecutors" + return newBaseURI + "/api/v1/applications/" + appId + "/allexecutors"; } ind = words.indexOf("history"); if (ind > 0) { diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.js b/core/src/main/resources/org/apache/spark/ui/static/webui.js index b1254e08fa50..89622106ff1f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.js +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.js @@ -33,7 +33,7 @@ function collapseTable(thisName, table){ var status = window.localStorage.getItem(thisName) == "true"; status = !status; - thisClass = '.' + thisName + var thisClass = '.' + thisName // Expand the list of additional metrics. var tableDiv = $(thisClass).parent().find('.' + table); @@ -63,6 +63,7 @@ $(function() { collapseTablePageLoad('collapse-aggregated-finishedDrivers','aggregated-finishedDrivers'); collapseTablePageLoad('collapse-aggregated-runtimeInformation','aggregated-runtimeInformation'); collapseTablePageLoad('collapse-aggregated-sparkProperties','aggregated-sparkProperties'); + collapseTablePageLoad('collapse-aggregated-hadoopProperties','aggregated-hadoopProperties'); collapseTablePageLoad('collapse-aggregated-systemProperties','aggregated-systemProperties'); collapseTablePageLoad('collapse-aggregated-classpathEntries','aggregated-classpathEntries'); collapseTablePageLoad('collapse-aggregated-activeJobs','aggregated-activeJobs'); diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index ab0ae55ed357..67f2c279dca5 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import scala.concurrent.Future import org.apache.spark.executor.ExecutorMetrics -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerId @@ -83,7 +83,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" private val timeoutIntervalMs = - sc.conf.getTimeAsMs("spark.storage.blockManagerTimeoutIntervalMs", "60s") + sc.conf.get(config.STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL) private val checkTimeoutIntervalMs = sc.conf.getTimeAsSeconds("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms") * 1000 diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1c4fa4bc6541..a8b8e9688ac1 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -322,11 +322,10 @@ private[spark] class MapOutputTrackerMaster( extends MapOutputTracker(conf) { // The size at which we use Broadcast to send the map output statuses to the executors - private val minSizeForBroadcast = - conf.getSizeAsBytes("spark.shuffle.mapOutput.minSizeForBroadcast", "512k").toInt + private val minSizeForBroadcast = conf.get(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST).toInt /** Whether to compute locality preferences for reduce tasks */ - private val shuffleLocalityEnabled = conf.getBoolean("spark.shuffle.reduceLocality.enabled", true) + private val shuffleLocalityEnabled = conf.get(SHUFFLE_REDUCE_LOCALITY_ENABLE) // Number of map and reduce tasks above which we do not assign preferred locations based on map // output sizes. We limit the size of jobs for which assign preferred locations as computing the @@ -353,7 +352,7 @@ private[spark] class MapOutputTrackerMaster( // Thread pool used for handling map output status requests. This is a separate thread pool // to ensure we don't block the normal dispatcher threads. private val threadpool: ThreadPoolExecutor = { - val numThreads = conf.getInt("spark.shuffle.mapOutput.dispatcher.numThreads", 8) + val numThreads = conf.get(SHUFFLE_MAPOUTPUT_DISPATCHER_NUM_THREADS) val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "map-output-dispatcher") for (i <- 0 until numThreads) { pool.execute(new MessageLoop) @@ -364,9 +363,9 @@ private[spark] class MapOutputTrackerMaster( // Make sure that we aren't going to exceed the max RPC message size by making sure // we use broadcast to send large map output statuses. if (minSizeForBroadcast > maxRpcMessageSize) { - val msg = s"spark.shuffle.mapOutput.minSizeForBroadcast ($minSizeForBroadcast bytes) must " + - s"be <= spark.rpc.message.maxSize ($maxRpcMessageSize bytes) to prevent sending an rpc " + - "message that is too large." + val msg = s"${SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST.key} ($minSizeForBroadcast bytes) " + + s"must be <= spark.rpc.message.maxSize ($maxRpcMessageSize bytes) to prevent sending an " + + "rpc message that is too large." logError(msg) throw new IllegalArgumentException(msg) } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 15783c952c23..c64fdc02efc7 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -29,6 +29,7 @@ import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.sasl.SecretKeyHolder import org.apache.spark.util.Utils @@ -56,17 +57,13 @@ private[spark] class SecurityManager( private val WILDCARD_ACL = "*" private val authOn = sparkConf.get(NETWORK_AUTH_ENABLED) - // keep spark.ui.acls.enable for backwards compatibility with 1.0 - private var aclsOn = - sparkConf.getBoolean("spark.acls.enable", sparkConf.getBoolean("spark.ui.acls.enable", false)) + private var aclsOn = sparkConf.get(ACLS_ENABLE) // admin acls should be set before view or modify acls - private var adminAcls: Set[String] = - stringToSet(sparkConf.get("spark.admin.acls", "")) + private var adminAcls: Set[String] = sparkConf.get(ADMIN_ACLS).toSet // admin group acls should be set before view or modify group acls - private var adminAclsGroups : Set[String] = - stringToSet(sparkConf.get("spark.admin.acls.groups", "")) + private var adminAclsGroups: Set[String] = sparkConf.get(ADMIN_ACLS_GROUPS).toSet private var viewAcls: Set[String] = _ @@ -82,11 +79,11 @@ private[spark] class SecurityManager( private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), Utils.getCurrentUserName()) - setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) - setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) + setViewAcls(defaultAclUsers, sparkConf.get(UI_VIEW_ACLS)) + setModifyAcls(defaultAclUsers, sparkConf.get(MODIFY_ACLS)) - setViewAclsGroups(sparkConf.get("spark.ui.view.acls.groups", "")); - setModifyAclsGroups(sparkConf.get("spark.modify.acls.groups", "")); + setViewAclsGroups(sparkConf.get(UI_VIEW_ACLS_GROUPS)) + setModifyAclsGroups(sparkConf.get(MODIFY_ACLS_GROUPS)) private var secretKey: String = _ logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + @@ -127,23 +124,16 @@ private[spark] class SecurityManager( opts } - /** - * Split a comma separated String, filter out any empty items, and return a Set of strings - */ - private def stringToSet(list: String): Set[String] = { - list.split(',').map(_.trim).filter(!_.isEmpty).toSet - } - /** * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. */ - def setViewAcls(defaultUsers: Set[String], allowedUsers: String) { - viewAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) + def setViewAcls(defaultUsers: Set[String], allowedUsers: Seq[String]) { + viewAcls = adminAcls ++ defaultUsers ++ allowedUsers logInfo("Changing view acls to: " + viewAcls.mkString(",")) } - def setViewAcls(defaultUser: String, allowedUsers: String) { + def setViewAcls(defaultUser: String, allowedUsers: Seq[String]) { setViewAcls(Set[String](defaultUser), allowedUsers) } @@ -151,8 +141,8 @@ private[spark] class SecurityManager( * Admin acls groups should be set before the view or modify acls groups. If you modify the admin * acls groups you should also set the view and modify acls groups again to pick up the changes. */ - def setViewAclsGroups(allowedUserGroups: String) { - viewAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + def setViewAclsGroups(allowedUserGroups: Seq[String]) { + viewAclsGroups = adminAclsGroups ++ allowedUserGroups logInfo("Changing view acls groups to: " + viewAclsGroups.mkString(",")) } @@ -179,8 +169,8 @@ private[spark] class SecurityManager( * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. */ - def setModifyAcls(defaultUsers: Set[String], allowedUsers: String) { - modifyAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) + def setModifyAcls(defaultUsers: Set[String], allowedUsers: Seq[String]) { + modifyAcls = adminAcls ++ defaultUsers ++ allowedUsers logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) } @@ -188,8 +178,8 @@ private[spark] class SecurityManager( * Admin acls groups should be set before the view or modify acls groups. If you modify the admin * acls groups you should also set the view and modify acls groups again to pick up the changes. */ - def setModifyAclsGroups(allowedUserGroups: String) { - modifyAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + def setModifyAclsGroups(allowedUserGroups: Seq[String]) { + modifyAclsGroups = adminAclsGroups ++ allowedUserGroups logInfo("Changing modify acls groups to: " + modifyAclsGroups.mkString(",")) } @@ -216,8 +206,8 @@ private[spark] class SecurityManager( * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. */ - def setAdminAcls(adminUsers: String) { - adminAcls = stringToSet(adminUsers) + def setAdminAcls(adminUsers: Seq[String]) { + adminAcls = adminUsers.toSet logInfo("Changing admin acls to: " + adminAcls.mkString(",")) } @@ -225,8 +215,8 @@ private[spark] class SecurityManager( * Admin acls groups should be set before the view or modify acls groups. If you modify the admin * acls groups you should also set the view and modify acls groups again to pick up the changes. */ - def setAdminAclsGroups(adminUserGroups: String) { - adminAclsGroups = stringToSet(adminUserGroups) + def setAdminAclsGroups(adminUserGroups: Seq[String]) { + adminAclsGroups = adminUserGroups.toSet logInfo("Changing admin acls groups to: " + adminAclsGroups.mkString(",")) } @@ -416,7 +406,7 @@ private[spark] object SecurityManager { val k8sRegex = "k8s.*".r val SPARK_AUTH_CONF = NETWORK_AUTH_ENABLED.key - val SPARK_AUTH_SECRET_CONF = "spark.authenticate.secret" + val SPARK_AUTH_SECRET_CONF = AUTH_SECRET.key // This is used to set auth secret to an executor's env variable. It should have the same // value as SPARK_AUTH_SECRET_CONF set in SparkConf val ENV_AUTH_SECRET = "_SPARK_AUTH_SECRET" diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index f27df505fa5f..b596be0885b1 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -28,6 +28,7 @@ import org.apache.avro.{Schema, SchemaNormalization} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils @@ -123,7 +124,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor") - set("spark.jars", jars.filter(_ != null).mkString(",")) + set(JARS, jars.filter(_ != null)) } /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ @@ -201,12 +202,12 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria */ def registerKryoClasses(classes: Array[Class[_]]): SparkConf = { val allClassNames = new LinkedHashSet[String]() - allClassNames ++= get("spark.kryo.classesToRegister", "").split(',').map(_.trim) + allClassNames ++= get(KRYO_CLASSES_TO_REGISTER).map(_.trim) .filter(!_.isEmpty) allClassNames ++= classes.map(_.getName) - set("spark.kryo.classesToRegister", allClassNames.mkString(",")) - set("spark.serializer", classOf[KryoSerializer].getName) + set(KRYO_CLASSES_TO_REGISTER, allClassNames.toSeq) + set(SERIALIZER, classOf[KryoSerializer].getName) this } @@ -532,7 +533,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } // Validate memory fractions - for (key <- Seq("spark.memory.fraction", "spark.memory.storageFraction")) { + for (key <- Seq(MEMORY_FRACTION.key, MEMORY_STORAGE_FRACTION.key)) { val value = getDouble(key, 0.5) if (value > 1 || value < 0) { throw new IllegalArgumentException(s"$key should be between 0 and 1 (was '$value').") @@ -547,20 +548,20 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria case "yarn-cluster" => logWarning(warning) set("spark.master", "yarn") - set("spark.submit.deployMode", "cluster") + set(SUBMIT_DEPLOY_MODE, "cluster") case "yarn-client" => logWarning(warning) set("spark.master", "yarn") - set("spark.submit.deployMode", "client") + set(SUBMIT_DEPLOY_MODE, "client") case _ => // Any other unexpected master will be checked when creating scheduler backend. } } - if (contains("spark.submit.deployMode")) { - get("spark.submit.deployMode") match { + if (contains(SUBMIT_DEPLOY_MODE)) { + get(SUBMIT_DEPLOY_MODE) match { case "cluster" | "client" => - case e => throw new SparkException("spark.submit.deployMode can only be \"cluster\" or " + - "\"client\".") + case e => throw new SparkException(s"${SUBMIT_DEPLOY_MODE.key} can only be " + + "\"cluster\" or \"client\".") } } @@ -594,7 +595,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria // If spark.executor.heartbeatInterval bigger than spark.network.timeout, // it will almost always cause ExecutorLostFailure. See SPARK-22754. require(executorTimeoutThresholdMs > executorHeartbeatIntervalMs, "The value of " + - s"spark.network.timeout=${executorTimeoutThresholdMs}ms must be no less than the value of " + + s"spark.network.timeout=${executorTimeoutThresholdMs}ms must be greater than the value of " + s"spark.executor.heartbeatInterval=${executorHeartbeatIntervalMs}ms.") } @@ -664,7 +665,7 @@ private[spark] object SparkConf extends Logging { AlternateConfig("spark.yarn.applicationMaster.waitTries", "1.3", // Translate old value to a duration, with 10s wait time per try. translation = s => s"${s.toLong * 10}s")), - "spark.reducer.maxSizeInFlight" -> Seq( + REDUCER_MAX_SIZE_IN_FLIGHT.key -> Seq( AlternateConfig("spark.reducer.maxMbInFlight", "1.4")), "spark.kryoserializer.buffer" -> Seq( AlternateConfig("spark.kryoserializer.buffer.mb", "1.4", @@ -675,9 +676,9 @@ private[spark] object SparkConf extends Logging { AlternateConfig("spark.shuffle.file.buffer.kb", "1.4")), EXECUTOR_LOGS_ROLLING_MAX_SIZE.key -> Seq( AlternateConfig("spark.executor.logs.rolling.size.maxBytes", "1.4")), - "spark.io.compression.snappy.blockSize" -> Seq( + IO_COMPRESSION_SNAPPY_BLOCKSIZE.key -> Seq( AlternateConfig("spark.io.compression.snappy.block.size", "1.4")), - "spark.io.compression.lz4.blockSize" -> Seq( + IO_COMPRESSION_LZ4_BLOCKSIZE.key -> Seq( AlternateConfig("spark.io.compression.lz4.block.size", "1.4")), "spark.rpc.numRetries" -> Seq( AlternateConfig("spark.akka.num.retries", "1.4")), diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3a1e1b931002..e0c0635d6cfa 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -46,6 +46,7 @@ import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ @@ -228,7 +229,7 @@ class SparkContext(config: SparkConf) extends Logging { def jars: Seq[String] = _jars def files: Seq[String] = _files def master: String = _conf.get("spark.master") - def deployMode: String = _conf.getOption("spark.submit.deployMode").getOrElse("client") + def deployMode: String = _conf.get(SUBMIT_DEPLOY_MODE) def appName: String = _conf.get("spark.app.name") private[spark] def isEventLogEnabled: Boolean = _conf.get(EVENT_LOG_ENABLED) @@ -440,7 +441,7 @@ class SparkContext(config: SparkConf) extends Logging { } _ui = - if (conf.getBoolean("spark.ui.enabled", true)) { + if (conf.get(UI_ENABLED)) { Some(SparkUI.create(Some(this), _statusStore, _conf, _env.securityManager, appName, "", startTime)) } else { @@ -510,7 +511,7 @@ class SparkContext(config: SparkConf) extends Logging { _applicationId = _taskScheduler.applicationId() _applicationAttemptId = taskScheduler.applicationAttemptId() _conf.set("spark.app.id", _applicationId) - if (_conf.getBoolean("spark.ui.reverseProxy", false)) { + if (_conf.get(UI_REVERSE_PROXY)) { System.setProperty("spark.ui.proxyBase", "/proxy/" + _applicationId) } _ui.foreach(_.setAppId(_applicationId)) @@ -2369,8 +2370,8 @@ class SparkContext(config: SparkConf) extends Logging { val schedulingMode = getSchedulingMode.toString val addedJarPaths = addedJars.keys.toSeq val addedFilePaths = addedFiles.keys.toSeq - val environmentDetails = SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, - addedFilePaths) + val environmentDetails = SparkEnv.environmentDetails(conf, hadoopConfiguration, + schedulingMode, addedJarPaths, addedFilePaths) val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails) listenerBus.post(environmentUpdate) } @@ -2639,7 +2640,7 @@ object SparkContext extends Logging { case SparkMasterRegex.LOCAL_N_REGEX(threads) => convertToInt(threads) case SparkMasterRegex.LOCAL_N_FAILURES_REGEX(threads, _) => convertToInt(threads) case "yarn" => - if (conf != null && conf.getOption("spark.submit.deployMode").contains("cluster")) { + if (conf != null && conf.get(SUBMIT_DEPLOY_MODE) == "cluster") { conf.getInt(DRIVER_CORES.key, 0) } else { 0 diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index ba5ed8ab1f30..36998d159616 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -21,15 +21,17 @@ import java.io.File import java.net.Socket import java.util.Locale +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.Properties import com.google.common.collect.MapMaker +import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ import org.apache.spark.memory.{MemoryManager, UnifiedMemoryManager} import org.apache.spark.metrics.MetricsSystem @@ -274,14 +276,13 @@ object SparkEnv extends Logging { } } - // Create an instance of the class named by the given SparkConf property, or defaultClassName + // Create an instance of the class named by the given SparkConf property // if the property is not set, possibly initializing it with our conf - def instantiateClassFromConf[T](propertyName: String, defaultClassName: String): T = { - instantiateClass[T](conf.get(propertyName, defaultClassName)) + def instantiateClassFromConf[T](propertyName: ConfigEntry[String]): T = { + instantiateClass[T](conf.get(propertyName)) } - val serializer = instantiateClassFromConf[Serializer]( - "spark.serializer", "org.apache.spark.serializer.JavaSerializer") + val serializer = instantiateClassFromConf[Serializer](SERIALIZER) logDebug(s"Using serializer: ${serializer.getClass}") val serializerManager = new SerializerManager(serializer, conf, ioEncryptionKey) @@ -317,7 +318,7 @@ object SparkEnv extends Logging { val shortShuffleMgrNames = Map( "sort" -> classOf[org.apache.spark.shuffle.sort.SortShuffleManager].getName, "tungsten-sort" -> classOf[org.apache.spark.shuffle.sort.SortShuffleManager].getName) - val shuffleMgrName = conf.get("spark.shuffle.manager", "sort") + val shuffleMgrName = conf.get(config.SHUFFLE_MANAGER) val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase(Locale.ROOT), shuffleMgrName) val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) @@ -401,6 +402,7 @@ object SparkEnv extends Logging { private[spark] def environmentDetails( conf: SparkConf, + hadoopConf: Configuration, schedulingMode: String, addedJars: Seq[String], addedFiles: Seq[String]): Map[String, Seq[(String, String)]] = { @@ -436,9 +438,13 @@ object SparkEnv extends Logging { val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted + // Add Hadoop properties, it will not ignore configs including in Spark. Some spark + // conf starting with "spark.hadoop" may overwrite it. + val hadoopProperties = hadoopConf.asScala.map(entry => (entry.getKey, entry.getValue)).toSeq Map[String, Seq[(String, String)]]( "JVM Information" -> jvmInformation, "Spark Properties" -> sparkProperties, + "Hadoop Properties" -> hadoopProperties, "System Properties" -> otherProperties, "Classpath Entries" -> classPaths) } 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 5ed5070558af..14ea289e5f90 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 @@ -39,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.security.SocketAuthHelper @@ -604,7 +605,7 @@ private[spark] class PythonAccumulatorV2( Utils.checkHost(serverHost) - val bufferSize = SparkEnv.get.conf.getInt("spark.buffer.size", 65536) + val bufferSize = SparkEnv.get.conf.get(BUFFER_SIZE) /** * We try to reuse a single Socket to transfer accumulator updates, as they are all added diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 5168e9330965..b7f14e062b43 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -27,7 +27,7 @@ import scala.collection.JavaConverters._ import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.EXECUTOR_CORES +import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES} import org.apache.spark.internal.config.Python._ import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -71,7 +71,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( require(funcs.length == argOffsets.length, "argOffsets should have the same length as funcs") private val conf = SparkEnv.get.conf - private val bufferSize = conf.getInt("spark.buffer.size", 65536) + private val bufferSize = conf.get(BUFFER_SIZE) private val reuseWorker = conf.get(PYTHON_WORKER_REUSE) // each python worker gets an equal part of the allocation. the worker pool will grow to the // number of concurrent tasks, which is determined by the number of cores in this executor. 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 index 3fdea04cdf7a..b367c7ffb159 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -27,6 +27,7 @@ import scala.util.Try import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.internal.config.R._ import org.apache.spark.util.Utils @@ -124,7 +125,8 @@ private[spark] class RRunner[U]( partitionIndex: Int): Unit = { val env = SparkEnv.get val taskContext = TaskContext.get() - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val bufferSize = System.getProperty(BUFFER_SIZE.key, + BUFFER_SIZE.defaultValueString).toInt val stream = new BufferedOutputStream(output, bufferSize) new Thread("writer for R") { diff --git a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala index 9bf35af1da92..6832223a5daf 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala @@ -23,6 +23,7 @@ import java.util.Arrays import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.api.python.PythonUtils +import org.apache.spark.internal.config._ private[spark] object RUtils { // Local path where R binary packages built from R source code contained in the spark @@ -63,7 +64,7 @@ private[spark] object RUtils { (sys.props("spark.master"), sys.props("spark.submit.deployMode")) } else { val sparkConf = SparkEnv.get.conf - (sparkConf.get("spark.master"), sparkConf.get("spark.submit.deployMode", "client")) + (sparkConf.get("spark.master"), sparkConf.get(SUBMIT_DEPLOY_MODE)) } val isYarnCluster = master != null && master.contains("yarn") && deployMode == "cluster" 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 26ead57316e1..6410866fca6a 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import scala.util.Random import org.apache.spark._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer import org.apache.spark.storage._ @@ -74,14 +74,14 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) @transient private var blockSize: Int = _ private def setConf(conf: SparkConf) { - compressionCodec = if (conf.getBoolean("spark.broadcast.compress", true)) { + compressionCodec = if (conf.get(config.BROADCAST_COMPRESS)) { Some(CompressionCodec.createCodec(conf)) } else { None } // Note: use getSizeAsKb (not bytes) to maintain compatibility if no units are provided - blockSize = conf.getSizeAsKb("spark.broadcast.blockSize", "4m").toInt * 1024 - checksumEnabled = conf.getBoolean("spark.broadcast.checksum", true) + blockSize = conf.get(config.BROADCAST_BLOCKSIZE).toInt * 1024 + checksumEnabled = conf.get(config.BROADCAST_CHECKSUM) } setConf(SparkEnv.get.conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 0679bdf7c707..a66243012041 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -60,7 +60,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} private object FaultToleranceTest extends App with Logging { private val conf = new SparkConf() - private val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + private val zkDir = conf.get(config.Deploy.ZOOKEEPER_DIRECTORY).getOrElse("/spark") private val masters = ListBuffer[TestMasterInfo]() private val workers = ListBuffer[TestWorkerInfo]() @@ -87,8 +87,8 @@ private object FaultToleranceTest extends App with Logging { terminateCluster() // Clear ZK directories in between tests (for speed purposes) - SparkCuratorUtil.deleteRecursive(zk, ZK_DIR + "/spark_leader") - SparkCuratorUtil.deleteRecursive(zk, ZK_DIR + "/master_status") + SparkCuratorUtil.deleteRecursive(zk, zkDir + "/spark_leader") + SparkCuratorUtil.deleteRecursive(zk, zkDir + "/master_status") } test("sanity-basic") { diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index be293f88a9d4..c1866b4c3606 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -51,8 +51,8 @@ class LocalSparkCluster( // Disable REST server on Master in this mode unless otherwise specified val _conf = conf.clone() - .setIfMissing("spark.master.rest.enabled", "false") - .set(config.SHUFFLE_SERVICE_ENABLED.key, "false") + .setIfMissing(config.MASTER_REST_SERVER_ENABLED, false) + .set(config.SHUFFLE_SERVICE_ENABLED, false) /* Start the Master */ val (rpcEnv, webUiPort, _) = Master.startRpcEnvAndEndpoint(localHostname, 0, 0, _conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala index 8247110940db..8118c01eb712 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala @@ -25,6 +25,7 @@ import org.apache.zookeeper.KeeperException import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy.ZOOKEEPER_URL private[spark] object SparkCuratorUtil extends Logging { @@ -35,7 +36,7 @@ private[spark] object SparkCuratorUtil extends Logging { def newClient( conf: SparkConf, - zkUrlConf: String = "spark.deploy.zookeeper.url"): CuratorFramework = { + zkUrlConf: String = ZOOKEEPER_URL.key): CuratorFramework = { val ZK_URL = conf.get(zkUrlConf) val zk = CuratorFrameworkFactory.newClient(ZK_URL, ZK_SESSION_TIMEOUT_MILLIS, ZK_CONNECTION_TIMEOUT_MILLIS, 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 937199273dab..a97d0721de7b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -38,6 +38,7 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.util.Utils /** @@ -442,7 +443,7 @@ private[spark] object SparkHadoopUtil { } } appendSparkHadoopConfigs(conf, hadoopConf) - val bufferSize = conf.get("spark.buffer.size", "65536") + val bufferSize = conf.get(BUFFER_SIZE).toString hadoopConf.set("io.file.buffer.size", bufferSize) } } 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 a4c65aeaae3f..d5e17ffb55d9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -50,6 +50,7 @@ import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.rest._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util._ @@ -436,7 +437,7 @@ private[spark] class SparkSubmit extends Logging { } if (localPyFiles != null) { - sparkConf.set("spark.submit.pyFiles", localPyFiles) + sparkConf.set(SUBMIT_PYTHON_FILES, localPyFiles.split(",").toSeq) } // In YARN mode for an R app, add the SparkR package archive and the R package @@ -536,14 +537,14 @@ private[spark] class SparkSubmit extends Logging { // Yarn only OptionAssigner(args.queue, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.queue"), - OptionAssigner(args.numExecutors, YARN, ALL_DEPLOY_MODES, - confKey = EXECUTOR_INSTANCES.key), OptionAssigner(args.pyFiles, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.pyFiles"), OptionAssigner(args.jars, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.jars"), OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.files"), OptionAssigner(args.archives, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.archives"), // Other options + OptionAssigner(args.numExecutors, YARN | KUBERNETES, ALL_DEPLOY_MODES, + confKey = EXECUTOR_INSTANCES.key), OptionAssigner(args.executorCores, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES, confKey = EXECUTOR_CORES.key), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | KUBERNETES, ALL_DEPLOY_MODES, @@ -613,11 +614,11 @@ private[spark] class SparkSubmit extends Logging { // For YARN cluster mode, the jar is already distributed on each node as "app.jar" // For python and R files, the primary resource is already distributed as a regular file if (!isYarnCluster && !args.isPython && !args.isR) { - var jars = sparkConf.getOption("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) + var jars = sparkConf.get(JARS) if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) } - sparkConf.set("spark.jars", jars.mkString(",")) + sparkConf.set(JARS, jars) } // In standalone cluster mode, use the REST client to submit the application (Spark 1.3+). @@ -680,7 +681,7 @@ private[spark] class SparkSubmit extends Logging { // Second argument is main class childArgs += (args.primaryResource, "") if (args.pyFiles != null) { - sparkConf.set("spark.submit.pyFiles", args.pyFiles) + sparkConf.set(SUBMIT_PYTHON_FILES, args.pyFiles.split(",").toSeq) } } else if (args.isR) { // Second argument is main class @@ -747,18 +748,17 @@ private[spark] class SparkSubmit extends Logging { // Resolve and format python file paths properly before adding them to the PYTHONPATH. // The resolving part is redundant in the case of --py-files, but necessary if the user // explicitly sets `spark.submit.pyFiles` in his/her default properties file. - sparkConf.getOption("spark.submit.pyFiles").foreach { pyFiles => - val resolvedPyFiles = Utils.resolveURIs(pyFiles) - val formattedPyFiles = if (!isYarnCluster && !isMesosCluster) { - PythonRunner.formatPaths(resolvedPyFiles).mkString(",") - } else { - // Ignoring formatting python path in yarn and mesos cluster mode, these two modes - // support dealing with remote python files, they could distribute and add python files - // locally. - resolvedPyFiles - } - sparkConf.set("spark.submit.pyFiles", formattedPyFiles) + val pyFiles = sparkConf.get(SUBMIT_PYTHON_FILES) + val resolvedPyFiles = Utils.resolveURIs(pyFiles.mkString(",")) + val formattedPyFiles = if (!isYarnCluster && !isMesosCluster) { + PythonRunner.formatPaths(resolvedPyFiles).mkString(",") + } else { + // Ignoring formatting python path in yarn and mesos cluster mode, these two modes + // support dealing with remote python files, they could distribute and add python files + // locally. + resolvedPyFiles } + sparkConf.set(SUBMIT_PYTHON_FILES, formattedPyFiles.split(",").toSeq) (childArgs, childClasspath, sparkConf, childMainClass) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 34facd5a58c4..9692d2a7e1ce 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -576,27 +576,26 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --kill SUBMISSION_ID If given, kills the driver specified. | --status SUBMISSION_ID If given, requests the status of the driver specified. | - | Spark standalone and Mesos only: + | Spark standalone, Mesos and Kubernetes only: | --total-executor-cores NUM Total cores for all executors. | - | Spark standalone and YARN only: - | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, - | or all available cores on the worker in standalone mode) + | Spark standalone, YARN and Kubernetes only: + | --executor-cores NUM Number of cores used by each executor. (Default: 1 in + | YARN and K8S modes, or all available cores on the worker + | in standalone mode). | - | YARN-only: - | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). + | Spark on YARN and Kubernetes only: | --num-executors NUM Number of executors to launch (Default: 2). | If dynamic allocation is enabled, the initial number of | executors will be at least NUM. + | --principal PRINCIPAL Principal to be used to login to KDC. + | --keytab KEYTAB The full path to the file that contains the keytab for the + | principal specified above. + | + | Spark on YARN only: + | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --archives ARCHIVES Comma separated list of archives to be extracted into the | working directory of each executor. - | --principal PRINCIPAL Principal to be used to login to KDC, while running on - | secure HDFS. - | --keytab KEYTAB The full path to the file that contains the keytab for the - | principal specified above. This keytab will be copied to - | the node running the Application Master via the Secure - | Distributed Cache, for renewing the login tickets and the - | delegation tokens periodically. """.stripMargin ) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 3c5648434fa6..33e89c393680 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -42,10 +42,11 @@ import org.fusesource.leveldbjni.internal.NativeDB import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{DRIVER_LOG_DFS_DIR, History} +import org.apache.spark.internal.config._ import org.apache.spark.internal.config.History._ import org.apache.spark.internal.config.Status._ import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.scheduler.ReplayListenerBus._ @@ -105,12 +106,12 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private val logDir = conf.get(History.HISTORY_LOG_DIR) - private val HISTORY_UI_ACLS_ENABLE = conf.get(History.UI_ACLS_ENABLE) - private val HISTORY_UI_ADMIN_ACLS = conf.get(History.UI_ADMIN_ACLS) - private val HISTORY_UI_ADMIN_ACLS_GROUPS = conf.get(History.UI_ADMIN_ACLS_GROUPS) - logInfo(s"History server ui acls " + (if (HISTORY_UI_ACLS_ENABLE) "enabled" else "disabled") + - "; users with admin permissions: " + HISTORY_UI_ADMIN_ACLS.toString + - "; groups with admin permissions" + HISTORY_UI_ADMIN_ACLS_GROUPS.toString) + private val historyUiAclsEnable = conf.get(History.HISTORY_SERVER_UI_ACLS_ENABLE) + private val historyUiAdminAcls = conf.get(History.HISTORY_SERVER_UI_ADMIN_ACLS) + private val historyUiAdminAclsGroups = conf.get(History.HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS) + logInfo(s"History server ui acls " + (if (historyUiAclsEnable) "enabled" else "disabled") + + "; users with admin permissions: " + historyUiAdminAcls.mkString(",") + + "; groups with admin permissions" + historyUiAdminAclsGroups.mkString(",")) private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) // Visible for testing @@ -314,6 +315,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getLastUpdatedTime(): Long = lastScanTime.get() + /** + * Split a comma separated String, filter out any empty items, and return a Sequence of strings + */ + private def stringToSeq(list: String): Seq[String] = { + list.split(',').map(_.trim).filter(!_.isEmpty) + } + override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { val app = try { load(appId) @@ -330,13 +338,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val conf = this.conf.clone() val secManager = new SecurityManager(conf) - secManager.setAcls(HISTORY_UI_ACLS_ENABLE) + secManager.setAcls(historyUiAclsEnable) // make sure to set admin acls before view acls so they are properly picked up - secManager.setAdminAcls(HISTORY_UI_ADMIN_ACLS + "," + attempt.adminAcls.getOrElse("")) - secManager.setViewAcls(attempt.info.sparkUser, attempt.viewAcls.getOrElse("")) - secManager.setAdminAclsGroups(HISTORY_UI_ADMIN_ACLS_GROUPS + "," + - attempt.adminAclsGroups.getOrElse("")) - secManager.setViewAclsGroups(attempt.viewAclsGroups.getOrElse("")) + secManager.setAdminAcls(historyUiAdminAcls ++ stringToSeq(attempt.adminAcls.getOrElse(""))) + secManager.setViewAcls(attempt.info.sparkUser, stringToSeq(attempt.viewAcls.getOrElse(""))) + secManager.setAdminAclsGroups(historyUiAdminAclsGroups ++ + stringToSeq(attempt.adminAclsGroups.getOrElse(""))) + secManager.setViewAclsGroups(stringToSeq(attempt.viewAclsGroups.getOrElse(""))) val kvstore = try { diskManager match { @@ -1187,11 +1195,16 @@ private[history] class AppListingListener( // Only parse the first env update, since any future changes don't have any effect on // the ACLs set for the UI. if (!gotEnvUpdate) { + def emptyStringToNone(strOption: Option[String]): Option[String] = strOption match { + case Some("") => None + case _ => strOption + } + val allProperties = event.environmentDetails("Spark Properties").toMap - attempt.viewAcls = allProperties.get("spark.ui.view.acls") - attempt.adminAcls = allProperties.get("spark.admin.acls") - attempt.viewAclsGroups = allProperties.get("spark.ui.view.acls.groups") - attempt.adminAclsGroups = allProperties.get("spark.admin.acls.groups") + attempt.viewAcls = emptyStringToNone(allProperties.get(UI_VIEW_ACLS.key)) + attempt.adminAcls = emptyStringToNone(allProperties.get(ADMIN_ACLS.key)) + attempt.viewAclsGroups = emptyStringToNone(allProperties.get(UI_VIEW_ACLS_GROUPS.key)) + attempt.adminAclsGroups = emptyStringToNone(allProperties.get(ADMIN_ACLS_GROUPS.key)) gotEnvUpdate = true checkProgress() diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index ff2ea3b843ee..7c9ce14c652c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -31,6 +31,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.History +import org.apache.spark.internal.config.UI._ import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -302,11 +303,10 @@ object HistoryServer extends Logging { config.set(SecurityManager.SPARK_AUTH_CONF, "false") } - if (config.getBoolean("spark.acls.enable", config.getBoolean("spark.ui.acls.enable", false))) { - logInfo("Either spark.acls.enable or spark.ui.acls.enable is configured, clearing it and " + - "only using spark.history.ui.acl.enable") - config.set("spark.acls.enable", "false") - config.set("spark.ui.acls.enable", "false") + if (config.get(ACLS_ENABLE)) { + logInfo(s"${ACLS_ENABLE.key} is configured, " + + s"clearing it and only using ${History.HISTORY_SERVER_UI_ACLS_ENABLE.key}") + config.set(ACLS_ENABLE, false) } new SecurityManager(config) 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 e1184248af46..b26da8a467fd 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 @@ -33,6 +33,10 @@ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.deploy.rest.StandaloneRestServer import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Deploy._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ import org.apache.spark.serializer.{JavaSerializer, Serializer} @@ -54,12 +58,12 @@ private[deploy] class Master( // For application IDs private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) - private val WORKER_TIMEOUT_MS = conf.getLong("spark.worker.timeout", 60) * 1000 - private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) - private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) - private val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) - private val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") - private val MAX_EXECUTOR_RETRIES = conf.getInt("spark.deploy.maxExecutorRetries", 10) + private val workerTimeoutMs = conf.get(WORKER_TIMEOUT) * 1000 + private val retainedApplications = conf.get(RETAINED_APPLICATIONS) + private val retainedDrivers = conf.get(RETAINED_DRIVERS) + private val reaperIterations = conf.get(REAPER_ITERATIONS) + private val recoveryMode = conf.get(RECOVERY_MODE) + private val maxExecutorRetries = conf.get(MAX_EXECUTOR_RETRIES) val workers = new HashSet[WorkerInfo] val idToApp = new HashMap[String, ApplicationInfo] @@ -111,17 +115,17 @@ private[deploy] class Master( // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - private val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) + private val spreadOutApps = conf.get(SPREAD_OUT_APPS) // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) - private val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) - val reverseProxy = conf.getBoolean("spark.ui.reverseProxy", false) + private val defaultCores = conf.get(DEFAULT_CORES) + val reverseProxy = conf.get(UI_REVERSE_PROXY) if (defaultCores < 1) { - throw new SparkException("spark.deploy.defaultCores must be positive") + throw new SparkException(s"${DEFAULT_CORES.key} must be positive") } // Alternative application submission gateway that is stable across Spark versions - private val restServerEnabled = conf.getBoolean("spark.master.rest.enabled", false) + private val restServerEnabled = conf.get(MASTER_REST_SERVER_ENABLED) private var restServer: Option[StandaloneRestServer] = None private var restServerBoundPort: Option[Int] = None @@ -140,7 +144,7 @@ private[deploy] class Master( webUi.bind() masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort if (reverseProxy) { - masterWebUiUrl = conf.get("spark.ui.reverseProxyUrl", masterWebUiUrl) + masterWebUiUrl = conf.get(UI_REVERSE_PROXY_URL).orElse(Some(masterWebUiUrl)).get webUi.addProxy() logInfo(s"Spark Master is acting as a reverse proxy. Master, Workers and " + s"Applications UIs are available at $masterWebUiUrl") @@ -149,10 +153,10 @@ private[deploy] class Master( override def run(): Unit = Utils.tryLogNonFatalError { self.send(CheckForWorkerTimeOut) } - }, 0, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) + }, 0, workerTimeoutMs, TimeUnit.MILLISECONDS) if (restServerEnabled) { - val port = conf.getInt("spark.master.rest.port", 6066) + val port = conf.get(MASTER_REST_SERVER_PORT) restServer = Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) } restServerBoundPort = restServer.map(_.start()) @@ -166,7 +170,7 @@ private[deploy] class Master( applicationMetricsSystem.getServletHandlers.foreach(webUi.attachHandler) val serializer = new JavaSerializer(conf) - val (persistenceEngine_, leaderElectionAgent_) = RECOVERY_MODE match { + val (persistenceEngine_, leaderElectionAgent_) = recoveryMode match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") val zkFactory = @@ -177,7 +181,7 @@ private[deploy] class Master( new FileSystemRecoveryModeFactory(conf, serializer) (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) case "CUSTOM" => - val clazz = Utils.classForName(conf.get("spark.deploy.recoveryMode.factory")) + val clazz = Utils.classForName(conf.get(RECOVERY_MODE_FACTORY)) val factory = clazz.getConstructor(classOf[SparkConf], classOf[Serializer]) .newInstance(conf, serializer) .asInstanceOf[StandaloneRecoveryModeFactory] @@ -231,7 +235,7 @@ private[deploy] class Master( override def run(): Unit = Utils.tryLogNonFatalError { self.send(CompleteRecovery) } - }, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) + }, workerTimeoutMs, TimeUnit.MILLISECONDS) } case CompleteRecovery => completeRecovery() @@ -309,8 +313,8 @@ private[deploy] class Master( // Important note: this code path is not exercised by tests, so be very careful when // changing this `if` condition. if (!normalExit - && appInfo.incrementRetryCount() >= MAX_EXECUTOR_RETRIES - && MAX_EXECUTOR_RETRIES >= 0) { // < 0 disables this application-killing path + && appInfo.incrementRetryCount() >= maxExecutorRetries + && maxExecutorRetries >= 0) { // < 0 disables this application-killing path val execs = appInfo.executors.values if (!execs.exists(_.state == ExecutorState.RUNNING)) { logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + @@ -868,8 +872,8 @@ private[deploy] class Master( endpointToApp -= app.driver addressToApp -= app.driver.address - if (completedApps.size >= RETAINED_APPLICATIONS) { - val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) + if (completedApps.size >= retainedApplications) { + val toRemove = math.max(retainedApplications / 10, 1) completedApps.take(toRemove).foreach { a => applicationMetricsSystem.removeSource(a.appSource) } @@ -987,14 +991,14 @@ private[deploy] class Master( private def timeOutDeadWorkers() { // Copy the workers into an array so we don't modify the hashset while iterating through it val currentTime = System.currentTimeMillis() - val toRemove = workers.filter(_.lastHeartbeat < currentTime - WORKER_TIMEOUT_MS).toArray + val toRemove = workers.filter(_.lastHeartbeat < currentTime - workerTimeoutMs).toArray for (worker <- toRemove) { if (worker.state != WorkerState.DEAD) { logWarning("Removing %s because we got no heartbeat in %d seconds".format( - worker.id, WORKER_TIMEOUT_MS / 1000)) - removeWorker(worker, s"Not receiving heartbeat for ${WORKER_TIMEOUT_MS / 1000} seconds") + worker.id, workerTimeoutMs / 1000)) + removeWorker(worker, s"Not receiving heartbeat for ${workerTimeoutMs / 1000} seconds") } else { - if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT_MS)) { + if (worker.lastHeartbeat < currentTime - ((reaperIterations + 1) * workerTimeoutMs)) { workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it } } @@ -1029,8 +1033,8 @@ private[deploy] class Master( case Some(driver) => logInfo(s"Removing driver: $driverId") drivers -= driver - if (completedDrivers.size >= RETAINED_DRIVERS) { - val toRemove = math.max(RETAINED_DRIVERS / 10, 1) + if (completedDrivers.size >= retainedDrivers) { + val toRemove = math.max(retainedDrivers / 10, 1) completedDrivers.trimStart(toRemove) } completedDrivers += driver 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 615d2533cf08..cd31bbdcfab5 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 @@ -21,6 +21,7 @@ import scala.annotation.tailrec import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.MASTER_UI_PORT import org.apache.spark.util.{IntParam, Utils} /** @@ -53,8 +54,8 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) exte // This mutates the SparkConf, so all accesses to it must be made after this line propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - if (conf.contains("spark.master.ui.port")) { - webUiPort = conf.get("spark.master.ui.port").toInt + if (conf.contains(MASTER_UI_PORT.key)) { + webUiPort = conf.get(MASTER_UI_PORT) } @tailrec diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index ffdd635be4f5..470798793cec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy.RECOVERY_DIRECTORY import org.apache.spark.serializer.Serializer /** @@ -52,11 +53,11 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serial private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serializer) extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { - val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") + val recoveryDir = conf.get(RECOVERY_DIRECTORY) def createPersistenceEngine(): PersistenceEngine = { - logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) - new FileSystemPersistenceEngine(RECOVERY_DIR, serializer) + logInfo("Persisting recovery state to directory: " + recoveryDir) + new FileSystemPersistenceEngine(recoveryDir, serializer) } def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 1e8dabfbe6b0..47f309144bdc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -23,11 +23,12 @@ import org.apache.curator.framework.recipes.leader.{LeaderLatch, LeaderLatchList import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy.ZOOKEEPER_DIRECTORY private[master] class ZooKeeperLeaderElectionAgent(val masterInstance: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { - val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val workingDir = conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark") + "/leader_election" private var zk: CuratorFramework = _ private var leaderLatch: LeaderLatch = _ @@ -38,7 +39,7 @@ private[master] class ZooKeeperLeaderElectionAgent(val masterInstance: LeaderEle private def start() { logInfo("Starting ZooKeeper LeaderElection agent") zk = SparkCuratorUtil.newClient(conf) - leaderLatch = new LeaderLatch(zk, WORKING_DIR) + leaderLatch = new LeaderLatch(zk, workingDir) leaderLatch.addListener(this) leaderLatch.start() } 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 af850e4871e5..73dd0de01796 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 @@ -28,6 +28,7 @@ import org.apache.zookeeper.CreateMode import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.serializer.Serializer @@ -35,22 +36,22 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer extends PersistenceEngine with Logging { - private val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + private val workingDir = conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark") + "/master_status" private val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) - SparkCuratorUtil.mkdir(zk, WORKING_DIR) + SparkCuratorUtil.mkdir(zk, workingDir) override def persist(name: String, obj: Object): Unit = { - serializeIntoFile(WORKING_DIR + "/" + name, obj) + serializeIntoFile(workingDir + "/" + name, obj) } override def unpersist(name: String): Unit = { - zk.delete().forPath(WORKING_DIR + "/" + name) + zk.delete().forPath(workingDir + "/" + name) } override def read[T: ClassTag](prefix: String): Seq[T] = { - zk.getChildren.forPath(WORKING_DIR).asScala + zk.getChildren.forPath(workingDir).asScala .filter(_.startsWith(prefix)).flatMap(deserializeFromFile[T]) } @@ -66,13 +67,13 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer } private def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { - val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) + val fileData = zk.getData().forPath(workingDir + "/" + filename) try { Some(serializer.newInstance().deserialize[T](ByteBuffer.wrap(fileData))) } catch { case e: Exception => logWarning("Exception while reading persisted file, deleting", e) - zk.delete().forPath(WORKING_DIR + "/" + filename) + zk.delete().forPath(workingDir + "/" + filename) None } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index e87b2240564b..be402ae24751 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.UI_KILL_ENABLED import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -34,7 +35,7 @@ class MasterWebUI( requestedPort, master.conf, name = "MasterUI") with Logging { val masterEndpointRef = master.self - val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) + val killEnabled = master.conf.get(UI_KILL_ENABLED) initialize() diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index c75e684df226..a70754c6e2c4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -146,7 +146,7 @@ private[rest] class StandaloneSubmitRequestServlet( // the driver. val masters = sparkProperties.get("spark.master") val (_, masterPort) = Utils.extractHostPortFromSparkUrl(masterUrl) - val masterRestPort = this.conf.getInt("spark.master.rest.port", 6066) + val masterRestPort = this.conf.get(config.MASTER_REST_SERVER_PORT) val updatedMasters = masters.map( _.replace(s":$masterRestPort", s":$masterPort")).getOrElse(masterUrl) val appArgs = request.appArgs diff --git a/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala b/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala index aec0f72feb3c..f3638533e1b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/KafkaTokenUtil.scala @@ -17,12 +17,13 @@ package org.apache.spark.deploy.security -import java.{ util => ju } +import java.{util => ju} import java.text.SimpleDateFormat import scala.util.control.NonFatal import org.apache.hadoop.io.Text +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.kafka.clients.CommonClientConfigs @@ -33,6 +34,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, S import org.apache.kafka.common.security.token.delegation.DelegationToken import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -45,6 +47,8 @@ private[spark] object KafkaTokenUtil extends Logging { } private[security] def obtainToken(sparkConf: SparkConf): (Token[_ <: TokenIdentifier], Long) = { + checkProxyUser() + val adminClient = AdminClient.create(createAdminClientProperties(sparkConf)) val createDelegationTokenOptions = new CreateDelegationTokenOptions() val createResult = adminClient.createDelegationToken(createDelegationTokenOptions) @@ -59,6 +63,14 @@ private[spark] object KafkaTokenUtil extends Logging { ), token.tokenInfo.expiryTimestamp) } + private[security] def checkProxyUser(): Unit = { + val currentUser = UserGroupInformation.getCurrentUser() + // Obtaining delegation token for proxy user is planned but not yet implemented + // See https://issues.apache.org/jira/browse/KAFKA-6945 + require(!SparkHadoopUtil.get.isProxyUser(currentUser), "Obtaining delegation token for proxy " + + "user is not yet supported.") + } + private[security] def createAdminClientProperties(sparkConf: SparkConf): ju.Properties = { val adminClientProperties = new ju.Properties diff --git a/core/src/main/scala/org/apache/spark/deploy/security/README.md b/core/src/main/scala/org/apache/spark/deploy/security/README.md new file mode 100644 index 000000000000..c3ef60a231f0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/README.md @@ -0,0 +1,249 @@ +# Delegation Token Handling In Spark + +This document aims to explain and demystify delegation tokens as they are used by Spark, since +this topic is generally a huge source of confusion. + + +## What are delegation tokens and why use them? + +Delegation tokens (DTs from now on) are authentication tokens used by some services to replace +Kerberos service tokens. Many services in the Hadoop ecosystem have support for DTs, since they +have some very desirable advantages over Kerberos tokens: + +* No need to distribute Kerberos credentials + +In a distributed application, distributing Kerberos credentials is tricky. Not all users have +keytabs, and when they do, it's generally frowned upon to distribute them over the network as +part of application data. + +DTs allow for a single place (e.g. the Spark driver) to require Kerberos credentials. That entity +can then distribute the DTs to other parts of the distributed application (e.g. Spark executors), +so they can authenticate to services. + +* A single token per service is used for authentication + +If Kerberos authentication were used, each client connection to a server would require a trip +to the KDC and generation of a service ticket. In a distributed system, the number of service +tickets can balloon pretty quickly when you think about the number of client processes (e.g. Spark +executors) vs. the number of service processes (e.g. HDFS DataNodes). That generates unnecessary +extra load on the KDC, and may even run into usage limits set up by the KDC admin. + +* DTs are only used for authentication + +DTs, unlike TGTs, can only be used to authenticate to the specific service for which they were +issued. You cannot use an existing DT to create new DTs or to create DTs for a different service. + +So in short, DTs are *not* Kerberos tokens. They are used by many services to replace Kerberos +authentication, or even other forms of authentication, although there is nothing (aside from +maybe implementation details) that ties them to Kerberos or any other authentication mechanism. + + +## Lifecycle of DTs + +DTs, unlike Kerberos tokens, are service-specific. There is no centralized location you contact +to create a DT for a service. So, the first step needed to get a DT is being able to authenticate +to the service in question. In the Hadoop ecosystem, that is generally done using Kerberos. + +This requires Kerberos credentials to be available somewhere for the application to use. The user +is generally responsible for providing those credentials, which is most commonly done by logging +in to the KDC (e.g. using "kinit"). That generates a (Kerberos) "token cache" containing a TGT +(ticket granting ticket), which can then be used to request service tickets. + +There are other ways of obtaining TGTs, but, ultimately, you need a TGT to bootstrap the process. + +Once a TGT is available, the target service's client library can then be used to authenticate +to the service using the Kerberos credentials, and request the creation of a delegation token. +This token can now be sent to other processes and used to authenticate to different daemons +belonging to that service. + +And thus the first drawback of DTs becomes apparent: you need service-specific logic to create and +use them. + +Spark implements a (somewhat) pluggable, internal DT creation API. Support for new services can be +added by implementing a `HadoopDelegationTokenProvider` that is then called by Spark when generating +delegation tokens for an application. Spark makes the DTs available to code by stashing them in the +`UserGroupInformation` credentials, and it's up to the DT provider and the respective client library +to agree on how to use those tokens. + +Once they are created, the semantics of how DTs operate are also service-specific. But, in general, +they try to follow the semantics of Kerberos tokens: + +* A "renewable period (equivalent to TGT's "lifetime") which is for how long the DT is valid + before it requires renewal. +* A "max lifetime" (equivalent to TGT's "renewable life") which is for how long the DT can be + renewed. + +Once the token reaches its "max lifetime", a new one needs to be created by contacting the +appropriate service, restarting the above process. + + +## DT Renewal, Renewers, and YARN + +This is the most confusing part of DT handling, and part of it is because much of the system was +designed with MapReduce, and later YARN, in mind. + +As seen above, DTs need to be renewed periodically until they finally expire for good. An example of +this is the default configuration of HDFS services: delegation tokens are valid for up to 7 days, +and need to be renewed every 24 hours. If 24 hours pass without the token being renewed, the token +cannot be used anymore. And the token cannot be renewed anymore after 7 days. + +This raises the question: who renews tokens? And for a long time the answer was YARN. + +When YARN applications are submitted, a set of DTs is also submitted with them. YARN takes care +of distributing these tokens to containers (using conventions set by the `UserGroupInformation` +API) and, also, keeping them renewed while the app is running. These tokens are used not just +by the application; they are also used by YARN itself to implement features like log collection +and aggregation. + +But this has a few caveats. + + +1. Who renews the tokens? + +This is handled mostly transparently by the Hadoop libraries in the case of YARN. Some services have +the concept of a token "renewer". This "renewer" is the name of the principal that is allowed to +renew the DT. When submitting to YARN, that will be the principal that the YARN service is running +as, which means that the client application needs to know that information. + +For other resource managers, the renewer mostly does not matter, since there is no service that +is doing the renewal. Except that it sometimes leaks into library code, such as in SPARK-20328. + + +2. What tokens are renewed? + +This is probably the biggest caveat. + +As discussed in the previous section, DTs are service-specific, and require service-specific +libraries for creation *and* renewal. This means that for YARN to be able to renew application +tokens, YARN needs: + +* The client libraries for all the services the application is using +* Information about how to connect to the services the application is using +* Permissions to connect to those services + +In reality, though, most of the time YARN has access to a single HDFS cluster, and that will be +the extent of its DT renewal features. Any other tokens sent to YARN will be distributed to +containers, but will not be renewed. + +This means that those tokens will expire way before their max lifetime, unless some other code +takes care of renewing them. + +Also, not all client libraries even implement token renewal. To use the example of a service +supported by Spark, the `renew()` method of HBase tokens is a no-op. So the only way to "renew" an +HBase token is to create a new one. + + +3. What happens when tokens expire for good? + +The final caveat is that DTs have a maximum life, regardless of renewal. And after that deadline +is met, you need to create new tokens to be able to connect to the services. That means you need +the ability to connect to the service without a delegation token, which requires some form of +authentication aside from DTs. + +This is especially important for long-running applications that run unsupervised. They must be +able to keep on going without having someone logging into a terminal and typing a password every +few days. + + +## DT Renewal in Spark + +Because of the issues explained above, Spark implements a different way of doing renewal. Spark's +solution is a compromise: it targets the lowest common denominator, which is services like HBase +that do not support actual token renewal. + +In Spark, DT "renewal" is enabled by giving the application a Kerberos keytab. A keytab is +basically your Kerberos password written into a plain text file, which is why it's so sensitive: +if anyone is able to get hold of that keytab file, they'll be able to authenticate to any service +as that user, for as long as the credentials stored in the keytab remain valid in the KDC. + +By having the keytab, Spark can indefinitely maintain a valid Kerberos TGT. + +With Kerberos credentials available, Spark will create new DTs for the configured services as old +ones expire. So Spark doesn't renew tokens as explained in the previous section: it will create new +tokens at every renewal interval instead, and distribute those tokens to executors. + +This also has another advantage on top of supporting services like HBase: it removes the dependency +on an external renewal service (like YARN). That way, Spark's renewal feature can be used with +resource managers that are not DT-aware, such as Mesos or Kubernetes, as long as the application +has access to a keytab. + + +## DTs and Proxy Users + +"Proxy users" is Hadoop-speak for impersonation. It allows user A to impersonate user B when +connecting to a service, if that service allows it. + +Spark allows impersonation when submitting applications, so that the whole application runs as +user B in the above example. + +Spark does not allow token renewal when impersonation is on. Impersonation was added in Spark +as a means for services (like Hive or Oozie) to start Spark applications on behalf of users. +That means that those services would provide the Spark launcher code with privileged credentials +and, potentially, user code that will run when the application starts. The user code is not +necessarily under control of the service. + +In that situation, the service credentials should never be made available to the Spark application, +since that would be tantamount to giving your service credentials to unprivileged users. + +The above also implies that running impersonated applications in client mode can be a security +concern, since arbitrary user code would have access to the same local content as the privileged +user. But unlike token renewal, Spark does not prevent that configuration from running. + +When impersonating, the Spark launcher will create DTs for the "proxy" user. In the example +used above, that means that when code authenticates to a service using the DTs, the authenticated +user will be "B", not "A". + +Note that "proxy user" is a very Hadoop-specific concept. It does not apply to OS users (which +is why the client-mode case is an issue) and to services that do not authenticate using Hadoop's +`UserGroupInformation` system. It is generally used in the context of YARN - since an application +submitted as a proxy user will run as that particular user in the YARN cluster, obeying any +Hadoop-to-local-OS-user mapping configured for the service. But the overall support should work +for connecting to other services even when YARN is not being used. + +Also, if writing a new DT provider in Spark, be aware that providers need to explicitly handle +impersonation. If a service does not support impersonation, the provider should either error out or +not generate tokens, depending on what makes more sense in the context. + + +## Externally Generated DTs + +Spark uses the `UserGroupInformation` API to manage the Hadoop credentials. That means that Spark +inherits the feature of loading DTs automatically from a file. The Hadoop classes will load the +token cache pointed at by the `HADOOP_TOKEN_FILE_LOCATION` environment variable, when it's defined. + +In this situation, Spark will not create DTs for the services that already have tokens in the +cache. It may try to get delegation tokens for other services if Kerberos credentials are also +provided. + +This feature is mostly used by services that start Spark on behalf of users. Regular users do not +generally use this feature, given it would require them to figure out how to get those tokens +outside of Spark. + + +## Limitations of DT support in Spark + +There are certain limitations to bear in mind when talking about DTs in Spark. + +The first one is that not all DTs actually expose their renewal period. This is generally a +service configuration that is not generally exposed to clients. For this reason, certain DT +providers cannot provide a renewal period to the Spark code, thus requiring that the service's +configuration is in some way synchronized with another one that does provide that information. + +The HDFS service, which is generally available when DTs are needed in the first place, provides +that information, so in general it's a good idea for all services using DTs to use the same +configuration as HDFS for the renewal period. + +The second one is that Spark doesn't always know what delegation tokens will be needed. For +example, when submitting an application in cluster mode without a keytab, the launcher needs +to create DTs without knowing what the application code will actually be doing. This means that +Spark will try to get as many delegation tokens as is possible based on the configuration +available. That means that if an HBase configuration is available to the launcher but the app +doesn't actually use HBase, a DT will still be generated. The user would have to explicitly +opt out of generating HBase tokens in that case. + +The third one is that it's hard to create DTs "as needed". Without being able to authenticate +to specific services, Spark cannot create DTs, which means that applications submitted in cluster +mode like the above need DTs to be created up front, instead of on demand. + +The advantage, though, is that user code does not need to worry about DTs, since Spark will handle +them transparently when the proper configuration is available. 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 a6d13d12fc28..8c2a907b8689 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 @@ -31,6 +31,7 @@ import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Worker.WORKER_DRIVER_TERMINATE_TIMEOUT import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{Clock, ShutdownHookManager, SystemClock, Utils} @@ -57,8 +58,7 @@ private[deploy] class DriverRunner( @volatile private[worker] var finalException: Option[Exception] = None // Timeout to wait for when trying to terminate a driver. - private val DRIVER_TERMINATE_TIMEOUT_MS = - conf.getTimeAsMs("spark.worker.driverTerminateTimeout", "10s") + private val driverTerminateTimeoutMs = conf.get(WORKER_DRIVER_TERMINATE_TIMEOUT) // Decoupled for testing def setClock(_clock: Clock): Unit = { @@ -122,7 +122,7 @@ private[deploy] class DriverRunner( killed = true synchronized { process.foreach { p => - val exitCode = Utils.terminateProcess(p, DRIVER_TERMINATE_TIMEOUT_MS) + val exitCode = Utils.terminateProcess(p, driverTerminateTimeoutMs) if (exitCode.isEmpty) { logWarning("Failed to terminate driver process: " + p + ". This process will likely be orphaned.") 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 dc6a3076a511..c74a95718d82 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 @@ -28,6 +28,7 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{ShutdownHookManager, Utils} import org.apache.spark.util.logging.FileAppender @@ -160,7 +161,7 @@ private[deploy] class ExecutorRunner( // Add webUI log urls val baseUrl = - if (conf.getBoolean("spark.ui.reverseProxy", false)) { + if (conf.get(UI_REVERSE_PROXY)) { s"/proxy/$workerId/logPage/?appId=$appId&executorId=$execId&logType=" } else { s"http://$publicAddress:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType=" 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 467df26c4735..115450b93563 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 @@ -38,6 +38,8 @@ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils} @@ -73,7 +75,7 @@ private[deploy] class Worker( // For worker and executor IDs private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 + private val HEARTBEAT_MILLIS = conf.get(WORKER_TIMEOUT) * 1000 / 4 // Model retries to connect to the master, after Hadoop's model. // The first six attempts to reconnect are in shorter intervals (between 5 and 15 seconds) @@ -92,13 +94,11 @@ private[deploy] class Worker( private val PROLONGED_REGISTRATION_RETRY_INTERVAL_SECONDS = (math.round(60 * REGISTRATION_RETRY_FUZZ_MULTIPLIER)) - private val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) + private val CLEANUP_ENABLED = conf.get(WORKER_CLEANUP_ENABLED) // How often worker will clean up old app folders - private val CLEANUP_INTERVAL_MILLIS = - conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 + private val CLEANUP_INTERVAL_MILLIS = conf.get(WORKER_CLEANUP_INTERVAL) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up - private val APP_DATA_RETENTION_SECONDS = - conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) + private val APP_DATA_RETENTION_SECONDS = conf.get(APP_DATA_RETENTION) // Whether or not cleanup the non-shuffle files on executor exits. private val CLEANUP_NON_SHUFFLE_FILES_ENABLED = @@ -110,8 +110,7 @@ private[deploy] class Worker( * Whether to use the master address in `masterRpcAddresses` if possible. If it's disabled, Worker * will just use the address received from Master. */ - private val preferConfiguredMasterAddress = - conf.getBoolean("spark.worker.preferConfiguredMasterAddress", false) + private val preferConfiguredMasterAddress = conf.get(PREFER_CONFIGURED_MASTER_ADDRESS) /** * The master address to connect in case of failure. When the connection is broken, worker will * use this address to connect. This is usually just one of `masterRpcAddresses`. However, when @@ -142,10 +141,8 @@ private[deploy] class Worker( val appDirectories = new HashMap[String, Seq[String]] val finishedApps = new HashSet[String] - val retainedExecutors = conf.getInt("spark.worker.ui.retainedExecutors", - WorkerWebUI.DEFAULT_RETAINED_EXECUTORS) - val retainedDrivers = conf.getInt("spark.worker.ui.retainedDrivers", - WorkerWebUI.DEFAULT_RETAINED_DRIVERS) + val retainedExecutors = conf.get(WORKER_UI_RETAINED_EXECUTORS) + val retainedDrivers = conf.get(WORKER_UI_RETAINED_DRIVERS) // The shuffle service is not actually started unless configured. private val shuffleService = if (externalShuffleServiceSupplier != null) { @@ -165,7 +162,7 @@ private[deploy] class Worker( private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) private val workerSource = new WorkerSource(this) - val reverseProxy = conf.getBoolean("spark.ui.reverseProxy", false) + val reverseProxy = conf.get(UI_REVERSE_PROXY) private var registerMasterFutures: Array[JFuture[_]] = null private var registrationRetryTimer: Option[JScheduledFuture[_]] = None 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 580281288b06..8c87708e960e 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 @@ -22,6 +22,7 @@ import java.lang.management.ManagementFactory import scala.annotation.tailrec import org.apache.spark.SparkConf +import org.apache.spark.internal.config.Worker._ import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** @@ -59,9 +60,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { // This mutates the SparkConf, so all accesses to it must be made after this line propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - if (conf.contains("spark.worker.ui.port")) { - webUiPort = conf.get("spark.worker.ui.port").toInt - } + conf.get(WORKER_UI_PORT).foreach { webUiPort = _ } checkWorkerMemory() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 54886955b98f..96980c3ff033 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -56,6 +56,4 @@ class WorkerWebUI( private[worker] object WorkerWebUI { val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - val DEFAULT_RETAINED_DRIVERS = 1000 - val DEFAULT_RETAINED_EXECUTORS = 1000 } 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 a30a501e5d4a..a3644e713065 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -431,7 +431,7 @@ private[spark] class Executor( if (freedMemory > 0 && !threwException) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId" - if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) { + if (conf.get(UNSAFE_EXCEPTION_ON_MEMORY_LEAK)) { throw new SparkException(errMsg) } else { logWarning(errMsg) @@ -442,7 +442,7 @@ private[spark] class Executor( val errMsg = s"${releasedLocks.size} block locks were not released by TID = $taskId:\n" + releasedLocks.mkString("[", ", ", "]") - if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false)) { + if (conf.get(STORAGE_EXCEPTION_PIN_LEAK)) { throw new SparkException(errMsg) } else { logInfo(errMsg) diff --git a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala new file mode 100644 index 000000000000..ceab957b3663 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala @@ -0,0 +1,68 @@ +/* + * 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.internal.config + +private[spark] object Deploy { + val RECOVERY_MODE = ConfigBuilder("spark.deploy.recoveryMode") + .stringConf + .createWithDefault("NONE") + + val RECOVERY_MODE_FACTORY = ConfigBuilder("spark.deploy.recoveryMode.factory") + .stringConf + .createWithDefault("") + + val RECOVERY_DIRECTORY = ConfigBuilder("spark.deploy.recoveryDirectory") + .stringConf + .createWithDefault("") + + val ZOOKEEPER_URL = ConfigBuilder("spark.deploy.zookeeper.url") + .doc(s"When `${RECOVERY_MODE.key}` is set to ZOOKEEPER, this " + + "configuration is used to set the zookeeper URL to connect to.") + .stringConf + .createOptional + + val ZOOKEEPER_DIRECTORY = ConfigBuilder("spark.deploy.zookeeper.dir") + .stringConf + .createOptional + + val RETAINED_APPLICATIONS = ConfigBuilder("spark.deploy.retainedApplications") + .intConf + .createWithDefault(200) + + val RETAINED_DRIVERS = ConfigBuilder("spark.deploy.retainedDrivers") + .intConf + .createWithDefault(200) + + val REAPER_ITERATIONS = ConfigBuilder("spark.dead.worker.persistence") + .intConf + .createWithDefault(15) + + val MAX_EXECUTOR_RETRIES = ConfigBuilder("spark.deploy.maxExecutorRetries") + .intConf + .createWithDefault(10) + + val SPREAD_OUT_APPS = ConfigBuilder("spark.deploy.spreadOut") + .booleanConf + .createWithDefault(true) + + val DEFAULT_CORES = ConfigBuilder("spark.deploy.defaultCores") + .intConf + .createWithDefault(Int.MaxValue) + + +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index f984dd385344..e7d25bfe33dc 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -88,17 +88,19 @@ private[spark] object History { val MAX_DRIVER_LOG_AGE_S = ConfigBuilder("spark.history.fs.driverlog.cleaner.maxAge") .fallbackConf(MAX_LOG_AGE_S) - val UI_ACLS_ENABLE = ConfigBuilder("spark.history.ui.acls.enable") + val HISTORY_SERVER_UI_ACLS_ENABLE = ConfigBuilder("spark.history.ui.acls.enable") .booleanConf .createWithDefault(false) - val UI_ADMIN_ACLS = ConfigBuilder("spark.history.ui.admin.acls") + val HISTORY_SERVER_UI_ADMIN_ACLS = ConfigBuilder("spark.history.ui.admin.acls") .stringConf - .createWithDefault("") + .toSequence + .createWithDefault(Nil) - val UI_ADMIN_ACLS_GROUPS = ConfigBuilder("spark.history.ui.admin.acls.groups") + val HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS = ConfigBuilder("spark.history.ui.admin.acls.groups") .stringConf - .createWithDefault("") + .toSequence + .createWithDefault(Nil) val NUM_REPLAY_THREADS = ConfigBuilder("spark.history.fs.numReplayThreads") .intConf diff --git a/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala new file mode 100644 index 000000000000..7873141440de --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala @@ -0,0 +1,57 @@ +/* + * 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.internal.config + +import org.apache.spark.network.util.ByteUnit + +private[spark] object Kryo { + + val KRYO_REGISTRATION_REQUIRED = ConfigBuilder("spark.kryo.registrationRequired") + .booleanConf + .createWithDefault(false) + + val KRYO_USER_REGISTRATORS = ConfigBuilder("spark.kryo.registrator") + .stringConf + .createOptional + + val KRYO_CLASSES_TO_REGISTER = ConfigBuilder("spark.kryo.classesToRegister") + .stringConf + .toSequence + .createWithDefault(Nil) + + val KRYO_USE_UNSAFE = ConfigBuilder("spark.kyro.unsafe") + .booleanConf + .createWithDefault(false) + + val KRYO_USE_POOL = ConfigBuilder("spark.kyro.pool") + .booleanConf + .createWithDefault(true) + + val KRYO_REFERENCE_TRACKING = ConfigBuilder("spark.kryo.referenceTracking") + .booleanConf + .createWithDefault(true) + + val KRYO_SERIALIZER_BUFFER_SIZE = ConfigBuilder("spark.kryoserializer.buffer") + .bytesConf(ByteUnit.KiB) + .createWithDefaultString("64k") + + val KRYO_SERIALIZER_MAX_BUFFER_SIZE = ConfigBuilder("spark.kryoserializer.buffer.max") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("64m") + +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/UI.scala b/core/src/main/scala/org/apache/spark/internal/config/UI.scala new file mode 100644 index 000000000000..6c04f0dd2bbb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/UI.scala @@ -0,0 +1,145 @@ +/* + * 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.internal.config + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.util.ByteUnit + +private[spark] object UI { + + val UI_SHOW_CONSOLE_PROGRESS = ConfigBuilder("spark.ui.showConsoleProgress") + .doc("When true, show the progress bar in the console.") + .booleanConf + .createWithDefault(false) + + val UI_CONSOLE_PROGRESS_UPDATE_INTERVAL = + ConfigBuilder("spark.ui.consoleProgress.update.interval") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(200) + + val UI_ENABLED = ConfigBuilder("spark.ui.enabled") + .doc("Whether to run the web UI for the Spark application.") + .booleanConf + .createWithDefault(true) + + val UI_PORT = ConfigBuilder("spark.ui.port") + .doc("Port for your application's dashboard, which shows memory and workload data.") + .intConf + .createWithDefault(4040) + + val UI_FILTERS = ConfigBuilder("spark.ui.filters") + .doc("Comma separated list of filter class names to apply to the Spark Web UI.") + .stringConf + .toSequence + .createWithDefault(Nil) + + val UI_ALLOW_FRAMING_FROM = ConfigBuilder("spark.ui.allowFramingFrom") + .stringConf + .createOptional + + val UI_REVERSE_PROXY = ConfigBuilder("spark.ui.reverseProxy") + .doc("Enable running Spark Master as reverse proxy for worker and application UIs. " + + "In this mode, Spark master will reverse proxy the worker and application UIs to enable " + + "access without requiring direct access to their hosts. Use it with caution, as worker " + + "and application UI will not be accessible directly, you will only be able to access them" + + "through spark master/proxy public URL. This setting affects all the workers and " + + "application UIs running in the cluster and must be set on all the workers, drivers " + + " and masters.") + .booleanConf + .createWithDefault(false) + + val UI_REVERSE_PROXY_URL = ConfigBuilder("spark.ui.reverseProxyUrl") + .doc("This is the URL where your proxy is running. This URL is for proxy which is running " + + "in front of Spark Master. This is useful when running proxy for authentication e.g. " + + "OAuth proxy. Make sure this is a complete URL including scheme (http/https) and port to " + + "reach your proxy.") + .stringConf + .createOptional + + val UI_KILL_ENABLED = ConfigBuilder("spark.ui.killEnabled") + .doc("Allows jobs and stages to be killed from the web UI.") + .booleanConf + .createWithDefault(true) + + val UI_THREAD_DUMPS_ENABLED = ConfigBuilder("spark.ui.threadDumpsEnabled") + .booleanConf + .createWithDefault(true) + + val UI_X_XSS_PROTECTION = ConfigBuilder("spark.ui.xXssProtection") + .doc("Value for HTTP X-XSS-Protection response header") + .stringConf + .createWithDefaultString("1; mode=block") + + val UI_X_CONTENT_TYPE_OPTIONS = ConfigBuilder("spark.ui.xContentTypeOptions.enabled") + .doc("Set to 'true' for setting X-Content-Type-Options HTTP response header to 'nosniff'") + .booleanConf + .createWithDefault(true) + + val UI_STRICT_TRANSPORT_SECURITY = ConfigBuilder("spark.ui.strictTransportSecurity") + .doc("Value for HTTP Strict Transport Security Response Header") + .stringConf + .createOptional + + val UI_REQUEST_HEADER_SIZE = ConfigBuilder("spark.ui.requestHeaderSize") + .doc("Value for HTTP request header size in bytes.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("8k") + + val UI_TIMELINE_TASKS_MAXIMUM = ConfigBuilder("spark.ui.timeline.tasks.maximum") + .intConf + .createWithDefault(1000) + + val ACLS_ENABLE = ConfigBuilder("spark.acls.enable") + .booleanConf + .createWithDefault(false) + + val UI_VIEW_ACLS = ConfigBuilder("spark.ui.view.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val UI_VIEW_ACLS_GROUPS = ConfigBuilder("spark.ui.view.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val ADMIN_ACLS = ConfigBuilder("spark.admin.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val ADMIN_ACLS_GROUPS = ConfigBuilder("spark.admin.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val MODIFY_ACLS = ConfigBuilder("spark.modify.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val MODIFY_ACLS_GROUPS = ConfigBuilder("spark.modify.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val USER_GROUPS_MAPPING = ConfigBuilder("spark.user.groups.mapping") + .stringConf + .createWithDefault("org.apache.spark.security.ShellBasedGroupsMappingProvider") +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/Worker.scala b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala new file mode 100644 index 000000000000..47f7167d2c9c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala @@ -0,0 +1,63 @@ +/* + * 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.internal.config + +import java.util.concurrent.TimeUnit + +private[spark] object Worker { + val WORKER_TIMEOUT = ConfigBuilder("spark.worker.timeout") + .longConf + .createWithDefault(60) + + val WORKER_DRIVER_TERMINATE_TIMEOUT = ConfigBuilder("spark.worker.driverTerminateTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("10s") + + val WORKER_CLEANUP_ENABLED = ConfigBuilder("spark.worker.cleanup.enabled") + .booleanConf + .createWithDefault(false) + + val WORKER_CLEANUP_INTERVAL = ConfigBuilder("spark.worker.cleanup.interval") + .longConf + .createWithDefault(60 * 30) + + val APP_DATA_RETENTION = ConfigBuilder("spark.worker.cleanup.appDataTtl") + .longConf + .createWithDefault(7 * 24 * 3600) + + val PREFER_CONFIGURED_MASTER_ADDRESS = ConfigBuilder("spark.worker.preferConfiguredMasterAddress") + .booleanConf + .createWithDefault(false) + + val WORKER_UI_PORT = ConfigBuilder("spark.worker.ui.port") + .intConf + .createOptional + + val WORKER_UI_RETAINED_EXECUTORS = ConfigBuilder("spark.worker.ui.retainedExecutors") + .intConf + .createWithDefault(1000) + + val WORKER_UI_RETAINED_DRIVERS = ConfigBuilder("spark.worker.ui.retainedDrivers") + .intConf + .createWithDefault(1000) + + val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = + ConfigBuilder("spark.worker.ui.compressedLogFileLengthCacheSize") + .intConf + .createWithDefault(100) +} 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 8caaa73b0227..1e7280005514 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 @@ -22,8 +22,10 @@ import java.util.concurrent.TimeUnit import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.EventLoggingListener +import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy} import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils +import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillReader.MAX_BUFFER_SIZE_BYTES package object config { @@ -187,6 +189,82 @@ package object config { .checkValue(_ >= 0, "The off-heap memory size must not be negative") .createWithDefault(0) + private[spark] val MEMORY_STORAGE_FRACTION = ConfigBuilder("spark.memory.storageFraction") + .doc("Amount of storage memory immune to eviction, expressed as a fraction of the " + + "size of the region set aside by spark.memory.fraction. The higher this is, the " + + "less working memory may be available to execution and tasks may spill to disk more " + + "often. Leaving this at the default value is recommended. ") + .doubleConf + .createWithDefault(0.5) + + private[spark] val MEMORY_FRACTION = ConfigBuilder("spark.memory.fraction") + .doc("Fraction of (heap space - 300MB) used for execution and storage. The " + + "lower this is, the more frequently spills and cached data eviction occur. " + + "The purpose of this config is to set aside memory for internal metadata, " + + "user data structures, and imprecise size estimation in the case of sparse, " + + "unusually large records. Leaving this at the default value is recommended. ") + .doubleConf + .createWithDefault(0.6) + + private[spark] val STORAGE_SAFETY_FRACTION = ConfigBuilder("spark.storage.safetyFraction") + .doubleConf + .createWithDefault(0.9) + + private[spark] val STORAGE_UNROLL_MEMORY_THRESHOLD = + ConfigBuilder("spark.storage.unrollMemoryThreshold") + .doc("Initial memory to request before unrolling any block") + .longConf + .createWithDefault(1024 * 1024) + + private[spark] val STORAGE_REPLICATION_PROACTIVE = + ConfigBuilder("spark.storage.replication.proactive") + .doc("Enables proactive block replication for RDD blocks. " + + "Cached RDD block replicas lost due to executor failures are replenished " + + "if there are any existing available replicas. This tries to " + + "get the replication level of the block to the initial number") + .booleanConf + .createWithDefault(false) + + private[spark] val STORAGE_MEMORY_MAP_THRESHOLD = + ConfigBuilder("spark.storage.memoryMapThreshold") + .doc("Size in bytes of a block above which Spark memory maps when " + + "reading a block from disk. " + + "This prevents Spark from memory mapping very small blocks. " + + "In general, memory mapping has high overhead for blocks close to or below " + + "the page size of the operating system.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("2m") + + private[spark] val STORAGE_REPLICATION_POLICY = + ConfigBuilder("spark.storage.replication.policy") + .stringConf + .createWithDefaultString(classOf[RandomBlockReplicationPolicy].getName) + + private[spark] val STORAGE_REPLICATION_TOPOLOGY_MAPPER = + ConfigBuilder("spark.storage.replication.topologyMapper") + .stringConf + .createWithDefaultString(classOf[DefaultTopologyMapper].getName) + + private[spark] val STORAGE_CACHED_PEERS_TTL = ConfigBuilder("spark.storage.cachedPeersTtl") + .intConf.createWithDefault(60 * 1000) + + private[spark] val STORAGE_MAX_REPLICATION_FAILURE = + ConfigBuilder("spark.storage.maxReplicationFailures") + .intConf.createWithDefault(1) + + private[spark] val STORAGE_REPLICATION_TOPOLOGY_FILE = + ConfigBuilder("spark.storage.replication.topologyFile").stringConf.createOptional + + private[spark] val STORAGE_EXCEPTION_PIN_LEAK = + ConfigBuilder("spark.storage.exceptionOnPinLeak") + .booleanConf + .createWithDefault(false) + + private[spark] val STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL = + ConfigBuilder("spark.storage.blockManagerTimeoutIntervalMs") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("60s") + private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() .booleanConf.createWithDefault(false) @@ -326,6 +404,10 @@ package object config { .stringConf .createOptional + private[spark] val METRICS_CONF = ConfigBuilder("spark.metrics.conf") + .stringConf + .createOptional + private[spark] val PYSPARK_DRIVER_PYTHON = ConfigBuilder("spark.pyspark.driver.python") .stringConf .createOptional @@ -338,11 +420,6 @@ package object config { private[spark] val HISTORY_UI_MAX_APPS = ConfigBuilder("spark.history.ui.maxApplications").intConf.createWithDefault(Integer.MAX_VALUE) - private[spark] val UI_SHOW_CONSOLE_PROGRESS = ConfigBuilder("spark.ui.showConsoleProgress") - .doc("When true, show the progress bar in the console.") - .booleanConf - .createWithDefault(false) - private[spark] val IO_ENCRYPTION_ENABLED = ConfigBuilder("spark.io.encryption.enabled") .booleanConf .createWithDefault(false) @@ -401,7 +478,7 @@ package object config { private[spark] val IGNORE_MISSING_FILES = ConfigBuilder("spark.files.ignoreMissingFiles") .doc("Whether to ignore missing files. If true, the Spark jobs will continue to run when " + - "encountering missing files and the contents that have been read will still be returned.") + "encountering missing files and the contents that have been read will still be returned.") .booleanConf .createWithDefault(false) @@ -436,7 +513,7 @@ package object config { "a property key or value, the value is redacted from the environment UI and various logs " + "like YARN and event logs.") .regexConf - .createWithDefault("(?i)secret|password".r) + .createWithDefault("(?i)secret|password|token".r) private[spark] val STRING_REDACTION_PATTERN = ConfigBuilder("spark.redaction.string.regex") @@ -446,6 +523,11 @@ package object config { .regexConf .createOptional + private[spark] val AUTH_SECRET = + ConfigBuilder("spark.authenticate.secret") + .stringConf + .createOptional + private[spark] val AUTH_SECRET_BIT_LENGTH = ConfigBuilder("spark.authenticate.secretBitLength") .intConf @@ -625,30 +707,6 @@ package object config { .toSequence .createWithDefault(Nil) - private[spark] val UI_X_XSS_PROTECTION = - ConfigBuilder("spark.ui.xXssProtection") - .doc("Value for HTTP X-XSS-Protection response header") - .stringConf - .createWithDefaultString("1; mode=block") - - private[spark] val UI_X_CONTENT_TYPE_OPTIONS = - ConfigBuilder("spark.ui.xContentTypeOptions.enabled") - .doc("Set to 'true' for setting X-Content-Type-Options HTTP response header to 'nosniff'") - .booleanConf - .createWithDefault(true) - - private[spark] val UI_STRICT_TRANSPORT_SECURITY = - ConfigBuilder("spark.ui.strictTransportSecurity") - .doc("Value for HTTP Strict Transport Security Response Header") - .stringConf - .createOptional - - private[spark] val UI_REQUEST_HEADER_SIZE = - ConfigBuilder("spark.ui.requestHeaderSize") - .doc("Value for HTTP request header size in bytes.") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("8k") - private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners") .doc("Class names of listeners to add to SparkContext during initialization.") .stringConf @@ -692,6 +750,96 @@ package object config { .timeConf(TimeUnit.SECONDS) .createWithDefaultString("1h") + private[spark] val SHUFFLE_SORT_INIT_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.sort.initialBufferSize") + .internal() + .intConf + .checkValue(v => v > 0, "The value should be a positive integer.") + .createWithDefault(4096) + + private[spark] val SHUFFLE_COMPRESS = + ConfigBuilder("spark.shuffle.compress") + .doc("Whether to compress shuffle output. Compression will use " + + "spark.io.compression.codec.") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_SPILL_COMPRESS = + ConfigBuilder("spark.shuffle.spill.compress") + .doc("Whether to compress data spilled during shuffles. Compression will use " + + "spark.io.compression.codec.") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_SPILL_INITIAL_MEM_THRESHOLD = + ConfigBuilder("spark.shuffle.spill.initialMemoryThreshold") + .internal() + .doc("Initial threshold for the size of a collection before we start tracking its " + + "memory usage.") + .longConf + .createWithDefault(5 * 1024 * 1024) + + private[spark] val SHUFFLE_SPILL_BATCH_SIZE = + ConfigBuilder("spark.shuffle.spill.batchSize") + .internal() + .doc("Size of object batches when reading/writing from serializers.") + .longConf + .createWithDefault(10000) + + private[spark] val SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD = + ConfigBuilder("spark.shuffle.sort.bypassMergeThreshold") + .doc("In the sort-based shuffle manager, avoid merge-sorting data if there is no " + + "map-side aggregation and there are at most this many reduce partitions") + .intConf + .createWithDefault(200) + + private[spark] val SHUFFLE_MANAGER = + ConfigBuilder("spark.shuffle.manager") + .stringConf + .createWithDefault("sort") + + private[spark] val SHUFFLE_REDUCE_LOCALITY_ENABLE = + ConfigBuilder("spark.shuffle.reduceLocality.enabled") + .doc("Whether to compute locality preferences for reduce tasks") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST = + ConfigBuilder("spark.shuffle.mapOutput.minSizeForBroadcast") + .doc("The size at which we use Broadcast to send the map output statuses to the executors.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("512k") + + private[spark] val SHUFFLE_MAPOUTPUT_DISPATCHER_NUM_THREADS = + ConfigBuilder("spark.shuffle.mapOutput.dispatcher.numThreads") + .intConf + .createWithDefault(8) + + private[spark] val SHUFFLE_DETECT_CORRUPT = + ConfigBuilder("spark.shuffle.detectCorrupt") + .doc("Whether to detect any corruption in fetched blocks.") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_SYNC = + ConfigBuilder("spark.shuffle.sync") + .doc("Whether to force outstanding writes to disk.") + .booleanConf + .createWithDefault(false) + + private[spark] val SHUFFLE_UNDAFE_FAST_MERGE_ENABLE = + ConfigBuilder("spark.shuffle.unsafe.fastMergeEnabled") + .doc("Whether to perform a fast spill merge.") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_SORT_USE_RADIXSORT = + ConfigBuilder("spark.shuffle.sort.useRadixSort") + .doc("Whether to use radix sort for sorting in-memory partition ids. Radix sort is much " + + "faster, but requires additional memory to be reserved memory as pointers are added.") + .booleanConf + .createWithDefault(true) + private[spark] val SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS = ConfigBuilder("spark.shuffle.minNumPartitionsToHighlyCompress") .internal() @@ -752,6 +900,26 @@ package object config { .checkValue(v => v > 0, "The max failures should be a positive value.") .createWithDefault(40) + private[spark] val UNSAFE_EXCEPTION_ON_MEMORY_LEAK = + ConfigBuilder("spark.unsafe.exceptionOnMemoryLeak") + .internal() + .booleanConf + .createWithDefault(false) + + private[spark] val UNSAFE_SORTER_SPILL_READ_AHEAD_ENABLED = + ConfigBuilder("spark.unsafe.sorter.spill.read.ahead.enabled") + .internal() + .booleanConf + .createWithDefault(true) + + private[spark] val UNSAFE_SORTER_SPILL_READER_BUFFER_SIZE = + ConfigBuilder("spark.unsafe.sorter.spill.reader.buffer.size") + .internal() + .bytesConf(ByteUnit.BYTE) + .checkValue(v => 1024 * 1024 <= v && v <= MAX_BUFFER_SIZE_BYTES, + s"The value must be in allowed range [1,048,576, ${MAX_BUFFER_SIZE_BYTES}].") + .createWithDefault(1024 * 1024) + private[spark] val EXECUTOR_PLUGINS = ConfigBuilder("spark.executor.plugins") .doc("Comma-separated list of class names for \"plugins\" implementing " + @@ -780,4 +948,159 @@ package object config { ConfigBuilder("spark.executor.logs.rolling.enableCompression") .booleanConf .createWithDefault(false) + + private[spark] val MASTER_REST_SERVER_ENABLED = ConfigBuilder("spark.master.rest.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val MASTER_REST_SERVER_PORT = ConfigBuilder("spark.master.rest.port") + .intConf + .createWithDefault(6066) + + private[spark] val MASTER_UI_PORT = ConfigBuilder("spark.master.ui.port") + .intConf + .createWithDefault(8080) + + private[spark] val IO_COMPRESSION_SNAPPY_BLOCKSIZE = + ConfigBuilder("spark.io.compression.snappy.blockSize") + .doc("Block size in bytes used in Snappy compression, in the case when " + + "Snappy compression codec is used. Lowering this block size " + + "will also lower shuffle memory usage when Snappy is used") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("32k") + + private[spark] val IO_COMPRESSION_LZ4_BLOCKSIZE = + ConfigBuilder("spark.io.compression.lz4.blockSize") + .doc("Block size in bytes used in LZ4 compression, in the case when LZ4 compression" + + "codec is used. Lowering this block size will also lower shuffle memory " + + "usage when LZ4 is used.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("32k") + + private[spark] val IO_COMPRESSION_CODEC = + ConfigBuilder("spark.io.compression.codec") + .doc("The codec used to compress internal data such as RDD partitions, event log, " + + "broadcast variables and shuffle outputs. By default, Spark provides four codecs: " + + "lz4, lzf, snappy, and zstd. You can also use fully qualified class names to specify " + + "the codec") + .stringConf + .createWithDefaultString("lz4") + + private[spark] val IO_COMPRESSION_ZSTD_BUFFERSIZE = + ConfigBuilder("spark.io.compression.zstd.bufferSize") + .doc("Buffer size in bytes used in Zstd compression, in the case when Zstd " + + "compression codec is used. Lowering this size will lower the shuffle " + + "memory usage when Zstd is used, but it might increase the compression " + + "cost because of excessive JNI call overhead") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("32k") + + private[spark] val IO_COMPRESSION_ZSTD_LEVEL = + ConfigBuilder("spark.io.compression.zstd.level") + .doc("Compression level for Zstd compression codec. Increasing the compression " + + "level will result in better compression at the expense of more CPU and memory") + .intConf + .createWithDefault(1) + + private[spark] val BUFFER_SIZE = + ConfigBuilder("spark.buffer.size") + .intConf + .createWithDefault(65536) + + private[spark] val LOCALITY_WAIT_PROCESS = ConfigBuilder("spark.locality.wait.process") + .fallbackConf(LOCALITY_WAIT) + + private[spark] val LOCALITY_WAIT_NODE = ConfigBuilder("spark.locality.wait.node") + .fallbackConf(LOCALITY_WAIT) + + private[spark] val LOCALITY_WAIT_RACK = ConfigBuilder("spark.locality.wait.rack") + .fallbackConf(LOCALITY_WAIT) + + private[spark] val REDUCER_MAX_SIZE_IN_FLIGHT = ConfigBuilder("spark.reducer.maxSizeInFlight") + .doc("Maximum size of map outputs to fetch simultaneously from each reduce task, " + + "in MiB unless otherwise specified. Since each output requires us to create a " + + "buffer to receive it, this represents a fixed memory overhead per reduce task, " + + "so keep it small unless you have a large amount of memory") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("48m") + + private[spark] val REDUCER_MAX_REQS_IN_FLIGHT = ConfigBuilder("spark.reducer.maxReqsInFlight") + .doc("This configuration limits the number of remote requests to fetch blocks at " + + "any given point. When the number of hosts in the cluster increase, " + + "it might lead to very large number of inbound connections to one or more nodes, " + + "causing the workers to fail under load. By allowing it to limit the number of " + + "fetch requests, this scenario can be mitigated") + .intConf + .createWithDefault(Int.MaxValue) + + private[spark] val BROADCAST_COMPRESS = ConfigBuilder("spark.broadcast.compress") + .doc("Whether to compress broadcast variables before sending them. " + + "Generally a good idea. Compression will use spark.io.compression.codec") + .booleanConf.createWithDefault(true) + + private[spark] val BROADCAST_BLOCKSIZE = ConfigBuilder("spark.broadcast.blockSize") + .doc("Size of each piece of a block for TorrentBroadcastFactory, in " + + "KiB unless otherwise specified. Too large a value decreases " + + "parallelism during broadcast (makes it slower); however, " + + "if it is too small, BlockManager might take a performance hit") + .bytesConf(ByteUnit.KiB) + .createWithDefaultString("4m") + + private[spark] val BROADCAST_CHECKSUM = ConfigBuilder("spark.broadcast.checksum") + .doc("Whether to enable checksum for broadcast. If enabled, " + + "broadcasts will include a checksum, which can help detect " + + "corrupted blocks, at the cost of computing and sending a little " + + "more data. It's possible to disable it if the network has other " + + "mechanisms to guarantee data won't be corrupted during broadcast") + .booleanConf.createWithDefault(true) + + private[spark] val RDD_COMPRESS = ConfigBuilder("spark.rdd.compress") + .doc("Whether to compress serialized RDD partitions " + + "(e.g. for StorageLevel.MEMORY_ONLY_SER in Scala " + + "or StorageLevel.MEMORY_ONLY in Python). Can save substantial " + + "space at the cost of some extra CPU time. " + + "Compression will use spark.io.compression.codec") + .booleanConf.createWithDefault(false) + + private[spark] val RDD_PARALLEL_LISTING_THRESHOLD = + ConfigBuilder("spark.rdd.parallelListingThreshold") + .intConf + .createWithDefault(10) + + private[spark] val RDD_LIMIT_SCALE_UP_FACTOR = + ConfigBuilder("spark.rdd.limit.scaleUpFactor") + .intConf + .createWithDefault(4) + + private[spark] val SERIALIZER = ConfigBuilder("spark.serializer") + .stringConf + .createWithDefault("org.apache.spark.serializer.JavaSerializer") + + private[spark] val SERIALIZER_OBJECT_STREAM_RESET = + ConfigBuilder("spark.serializer.objectStreamReset") + .intConf + .createWithDefault(100) + + private[spark] val SERIALIZER_EXTRA_DEBUG_INFO = ConfigBuilder("spark.serializer.extraDebugInfo") + .booleanConf + .createWithDefault(true) + + private[spark] val JARS = ConfigBuilder("spark.jars") + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val FILES = ConfigBuilder("spark.files") + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val SUBMIT_DEPLOY_MODE = ConfigBuilder("spark.submit.deployMode") + .stringConf + .createWithDefault("client") + + private[spark] val SUBMIT_PYTHON_FILES = ConfigBuilder("spark.submit.pyFiles") + .stringConf + .toSequence + .createWithDefault(Nil) } diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index c4f4b18769d2..288c0d18191c 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -27,6 +27,7 @@ import org.xerial.snappy.{Snappy, SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.config._ import org.apache.spark.util.Utils /** @@ -51,7 +52,7 @@ trait CompressionCodec { private[spark] object CompressionCodec { - private val configKey = "spark.io.compression.codec" + private val configKey = IO_COMPRESSION_CODEC.key private[spark] def supportsConcatenationOfSerializedStreams(codec: CompressionCodec): Boolean = { (codec.isInstanceOf[SnappyCompressionCodec] || codec.isInstanceOf[LZFCompressionCodec] @@ -65,7 +66,7 @@ private[spark] object CompressionCodec { "zstd" -> classOf[ZStdCompressionCodec].getName) def getCodecName(conf: SparkConf): String = { - conf.get(configKey, DEFAULT_COMPRESSION_CODEC) + conf.get(IO_COMPRESSION_CODEC) } def createCodec(conf: SparkConf): CompressionCodec = { @@ -117,7 +118,7 @@ private[spark] object CompressionCodec { class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getSizeAsBytes("spark.io.compression.lz4.blockSize", "32k").toInt + val blockSize = conf.get(IO_COMPRESSION_LZ4_BLOCKSIZE).toInt new LZ4BlockOutputStream(s, blockSize) } @@ -166,7 +167,7 @@ class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { } override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt + val blockSize = conf.get(IO_COMPRESSION_SNAPPY_BLOCKSIZE).toInt new SnappyOutputStream(s, blockSize) } @@ -185,10 +186,10 @@ class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { @DeveloperApi class ZStdCompressionCodec(conf: SparkConf) extends CompressionCodec { - private val bufferSize = conf.getSizeAsBytes("spark.io.compression.zstd.bufferSize", "32k").toInt + private val bufferSize = conf.get(IO_COMPRESSION_ZSTD_BUFFERSIZE).toInt // Default compression level for zstd compression to 1 because it is // fastest of all with reasonably high compression ratio. - private val level = conf.getInt("spark.io.compression.zstd.level", 1) + private val level = conf.get(IO_COMPRESSION_ZSTD_LEVEL) override def compressedOutputStream(s: OutputStream): OutputStream = { // Wrap the zstd output stream in a buffered output stream, so that we can 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 4fde2d0beaa7..4ba41072c3cc 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -57,7 +57,7 @@ private[spark] abstract class MemoryManager( protected[this] val maxOffHeapMemory = conf.get(MEMORY_OFFHEAP_SIZE) protected[this] val offHeapStorageMemory = - (maxOffHeapMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong + (maxOffHeapMemory * conf.get(MEMORY_STORAGE_FRACTION)).toLong offHeapExecutionMemoryPool.incrementPoolSize(maxOffHeapMemory - offHeapStorageMemory) offHeapStorageMemoryPool.incrementPoolSize(offHeapStorageMemory) diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index a0fbbbdebd02..7282a83f0739 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -203,7 +203,7 @@ object UnifiedMemoryManager { conf, maxHeapMemory = maxMemory, onHeapStorageRegionSize = - (maxMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong, + (maxMemory * conf.get(config.MEMORY_STORAGE_FRACTION)).toLong, numCores = numCores) } @@ -230,7 +230,7 @@ object UnifiedMemoryManager { } } val usableMemory = systemMemory - reservedMemory - val memoryFraction = conf.getDouble("spark.memory.fraction", 0.6) + val memoryFraction = conf.get(config.MEMORY_FRACTION) (usableMemory * memoryFraction).toLong } } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index a4056508c181..b6be8aaefd35 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -26,6 +26,7 @@ import scala.util.matching.Regex import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.METRICS_CONF import org.apache.spark.util.Utils private[spark] class MetricsConfig(conf: SparkConf) extends Logging { @@ -52,7 +53,7 @@ private[spark] class MetricsConfig(conf: SparkConf) extends Logging { // Add default properties in case there's no properties file setDefaultProperties(properties) - loadPropertiesFromFile(conf.getOption("spark.metrics.conf")) + loadPropertiesFromFile(conf.get(METRICS_CONF)) // Also look for the properties in provided Spark configuration val prefix = "spark.metrics.conf." diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 7076701421e2..27f4f94ea55f 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -59,7 +59,8 @@ class NettyBlockRpcServer( val blocksNum = openBlocks.blockIds.length val blocks = for (i <- (0 until blocksNum).view) yield blockManager.getBlockData(BlockId.apply(openBlocks.blockIds(i))) - val streamId = streamManager.registerStream(appId, blocks.iterator.asJava) + val streamId = streamManager.registerStream(appId, blocks.iterator.asJava, + client.getChannel) logTrace(s"Registered streamId $streamId with $blocksNum buffers") responseContext.onSuccess(new StreamHandle(streamId, blocksNum).toByteBuffer) 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 6a25ee20b2c6..b2ed2d30aca9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -36,6 +36,7 @@ import org.apache.spark.Partitioner._ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.RDD_LIMIT_SCALE_UP_FACTOR import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator @@ -1349,7 +1350,7 @@ abstract class RDD[T: ClassTag]( * an exception if called on an RDD of `Nothing` or `Null`. */ def take(num: Int): Array[T] = withScope { - val scaleUpFactor = Math.max(conf.getInt("spark.rdd.limit.scaleUpFactor", 4), 2) + val scaleUpFactor = Math.max(conf.get(RDD_LIMIT_SCALE_UP_FACTOR), 2) if (num == 0) { new Array[T](0) } else { diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala index 8273d8a9eb47..d165610291f1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.CHECKPOINT_COMPRESS +import org.apache.spark.internal.config.{BUFFER_SIZE, CHECKPOINT_COMPRESS} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -176,7 +176,7 @@ private[spark] object ReliableCheckpointRDD extends Logging { val tempOutputPath = new Path(outputDir, s".$finalOutputName-attempt-${ctx.attemptNumber()}") - val bufferSize = env.conf.getInt("spark.buffer.size", 65536) + val bufferSize = env.conf.get(BUFFER_SIZE) val fileOutputStream = if (blockSize < 0) { val fileStream = fs.create(tempOutputPath, false, bufferSize) @@ -222,7 +222,7 @@ private[spark] object ReliableCheckpointRDD extends Logging { sc: SparkContext, partitioner: Partitioner, checkpointDirPath: Path): Unit = { try { val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName) - val bufferSize = sc.conf.getInt("spark.buffer.size", 65536) + val bufferSize = sc.conf.get(BUFFER_SIZE) val fs = partitionerFilePath.getFileSystem(sc.hadoopConfiguration) val fileOutputStream = fs.create(partitionerFilePath, false, bufferSize) val serializer = SparkEnv.get.serializer.newInstance() @@ -249,7 +249,7 @@ private[spark] object ReliableCheckpointRDD extends Logging { sc: SparkContext, checkpointDirPath: String): Option[Partitioner] = { try { - val bufferSize = sc.conf.getInt("spark.buffer.size", 65536) + val bufferSize = sc.conf.get(BUFFER_SIZE) val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName) val fs = partitionerFilePath.getFileSystem(sc.hadoopConfiguration) val fileInputStream = fs.open(partitionerFilePath, bufferSize) @@ -287,7 +287,7 @@ private[spark] object ReliableCheckpointRDD extends Logging { context: TaskContext): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(broadcastedConf.value.value) - val bufferSize = env.conf.getInt("spark.buffer.size", 65536) + val bufferSize = env.conf.get(BUFFER_SIZE) val fileInputStream = { val fileStream = fs.open(path, bufferSize) if (env.conf.get(CHECKPOINT_COMPRESS)) { diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 60e383afadf1..6480e87c477a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -26,6 +26,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.util.Utils /** @@ -71,7 +72,7 @@ class UnionRDD[T: ClassTag]( // visible for testing private[spark] val isPartitionListingParallel: Boolean = - rdds.length > conf.getInt("spark.rdd.parallelListingThreshold", 10) + rdds.length > conf.get(RDD_PARALLEL_LISTING_THRESHOLD) override def getPartitions: Array[Partition] = { val parRDDs = if (isPartitionListingParallel) { 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 41f032ccf82b..1b42cb4d8207 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -1043,16 +1043,15 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.get(config.LOCALITY_WAIT) - val localityWaitKey = level match { - case TaskLocality.PROCESS_LOCAL => "spark.locality.wait.process" - case TaskLocality.NODE_LOCAL => "spark.locality.wait.node" - case TaskLocality.RACK_LOCAL => "spark.locality.wait.rack" + val localityWait = level match { + case TaskLocality.PROCESS_LOCAL => config.LOCALITY_WAIT_PROCESS + case TaskLocality.NODE_LOCAL => config.LOCALITY_WAIT_NODE + case TaskLocality.RACK_LOCAL => config.LOCALITY_WAIT_RACK case _ => null } - if (localityWaitKey != null) { - conf.getTimeAsMs(localityWaitKey, defaultWait.toString) + if (localityWait != null) { + conf.get(localityWait) } else { 0L } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 66080b6e6b4f..e0605fee9cbf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -224,8 +224,9 @@ private[spark] class StandaloneSchedulerBackend( if (stopping.compareAndSet(false, true)) { try { super.stop() - client.stop() - + if (client != null) { + client.stop() + } val callback = shutdownCallback if (callback != null) { callback(this) diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index f60dcfddfdc2..70564eeefda8 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.config._ import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} private[spark] class JavaSerializationStream( @@ -137,8 +138,8 @@ private[spark] class JavaSerializerInstance( */ @DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { - private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 100) - private var extraDebugInfo = conf.getBoolean("spark.serializer.extraDebugInfo", true) + private var counterReset = conf.get(SERIALIZER_OBJECT_STREAM_RESET) + private var extraDebugInfo = conf.get(SERIALIZER_EXTRA_DEBUG_INFO) protected def this() = this(new SparkConf()) // For deserialization only 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 72ca0fbe667e..2df133dd2b13 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -39,6 +39,7 @@ import org.roaringbitmap.RoaringBitmap import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ @@ -58,34 +59,34 @@ class KryoSerializer(conf: SparkConf) with Logging with Serializable { - private val bufferSizeKb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k") + private val bufferSizeKb = conf.get(KRYO_SERIALIZER_BUFFER_SIZE) if (bufferSizeKb >= ByteUnit.GiB.toKiB(2)) { - throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " + + throw new IllegalArgumentException(s"${KRYO_SERIALIZER_BUFFER_SIZE.key} must be less than " + s"2048 MiB, got: + ${ByteUnit.KiB.toMiB(bufferSizeKb)} MiB.") } private val bufferSize = ByteUnit.KiB.toBytes(bufferSizeKb).toInt - val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt + val maxBufferSizeMb = conf.get(KRYO_SERIALIZER_MAX_BUFFER_SIZE).toInt if (maxBufferSizeMb >= ByteUnit.GiB.toMiB(2)) { - throw new IllegalArgumentException("spark.kryoserializer.buffer.max must be less than " + - s"2048 MiB, got: + $maxBufferSizeMb MiB.") + throw new IllegalArgumentException(s"${KRYO_SERIALIZER_MAX_BUFFER_SIZE.key} must be less " + + s"than 2048 MiB, got: $maxBufferSizeMb MiB.") } private val maxBufferSize = ByteUnit.MiB.toBytes(maxBufferSizeMb).toInt - private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) - private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) - private val userRegistrators = conf.get("spark.kryo.registrator", "") - .split(',').map(_.trim) + private val referenceTracking = conf.get(KRYO_REFERENCE_TRACKING) + private val registrationRequired = conf.get(KRYO_REGISTRATION_REQUIRED) + private val userRegistrators = conf.get(KRYO_USER_REGISTRATORS) + .map(_.trim) .filter(!_.isEmpty) - private val classesToRegister = conf.get("spark.kryo.classesToRegister", "") - .split(',').map(_.trim) + private val classesToRegister = conf.get(KRYO_CLASSES_TO_REGISTER) + .map(_.trim) .filter(!_.isEmpty) private val avroSchemas = conf.getAvroSchema // whether to use unsafe based IO for serialization - private val useUnsafe = conf.getBoolean("spark.kryo.unsafe", false) - private val usePool = conf.getBoolean("spark.kryo.pool", true) + private val useUnsafe = conf.get(KRYO_USE_UNSAFE) + private val usePool = conf.get(KRYO_USE_POOL) def newKryoOutput(): KryoOutput = if (useUnsafe) { @@ -407,7 +408,7 @@ private[spark] class KryoSerializerInstance( } catch { case e: KryoException if e.getMessage.startsWith("Buffer overflow") => throw new SparkException(s"Kryo serialization failed: ${e.getMessage}. To avoid this, " + - "increase spark.kryoserializer.buffer.max value.", e) + s"increase ${KRYO_SERIALIZER_MAX_BUFFER_SIZE.key} value.", e) } finally { releaseKryo(kryo) } 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 1d4b05caaa14..3e3c387911d3 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -23,6 +23,7 @@ import java.nio.ByteBuffer import scala.reflect.ClassTag import org.apache.spark.SparkConf +import org.apache.spark.internal.config import org.apache.spark.io.CompressionCodec import org.apache.spark.security.CryptoStreamUtils import org.apache.spark.storage._ @@ -63,13 +64,13 @@ private[spark] class SerializerManager( } // Whether to compress broadcast variables that are stored - private[this] val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) + private[this] val compressBroadcast = conf.get(config.BROADCAST_COMPRESS) // Whether to compress shuffle output that are stored - private[this] val compressShuffle = conf.getBoolean("spark.shuffle.compress", true) + private[this] val compressShuffle = conf.get(config.SHUFFLE_COMPRESS) // Whether to compress RDD partitions that are stored serialized - private[this] val compressRdds = conf.getBoolean("spark.rdd.compress", false) + private[this] val compressRdds = conf.get(config.RDD_COMPRESS) // Whether to compress shuffle output temporarily spilled to disk - private[this] val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) + private[this] val compressShuffleSpill = conf.get(config.SHUFFLE_SPILL_COMPRESS) /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay * the initialization of the compression codec until it is first used. The reason is that a Spark 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 27e2f98c58f0..daafe305c8f8 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -50,11 +50,11 @@ private[spark] class BlockStoreShuffleReader[K, C]( mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), serializerManager.wrapStream, // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility - SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024, - SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue), + SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, + SparkEnv.get.conf.get(config.REDUCER_MAX_REQS_IN_FLIGHT), SparkEnv.get.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS), SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM), - SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true), + SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT), readMetrics) val serializerInstance = dep.serializer.newInstance() diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 274399b9cc1f..16058de8bf3f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -18,7 +18,7 @@ package org.apache.spark.shuffle.sort import org.apache.spark._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} import org.apache.spark.storage.ShuffleBlockId @@ -108,7 +108,7 @@ private[spark] object SortShuffleWriter { if (dep.mapSideCombine) { false } else { - val bypassMergeThreshold: Int = conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + val bypassMergeThreshold: Int = conf.get(config.SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD) dep.partitioner.numPartitions <= bypassMergeThreshold } } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 262ff6547faa..12c5d4d2c6ef 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -147,6 +147,7 @@ private[spark] class AppStatusListener( val envInfo = new v1.ApplicationEnvironmentInfo( runtime, details.getOrElse("Spark Properties", Nil), + details.getOrElse("Hadoop Properties", Nil), details.getOrElse("System Properties", Nil), details.getOrElse("Classpath Entries", Nil)) 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 c7d3cd37db6f..825fc5427806 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 @@ -352,6 +352,7 @@ class VersionInfo private[spark]( class ApplicationEnvironmentInfo private[spark] ( val runtime: RuntimeInfo, val sparkProperties: Seq[(String, String)], + val hadoopProperties: Seq[(String, String)], val systemProperties: Seq[(String, String)], val classpathEntries: Seq[(String, 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 1dfbc6effb34..8f993bfbf08a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -233,8 +233,7 @@ private[spark] class BlockManager( shuffleClient.init(appId) blockReplicationPolicy = { - val priorityClass = conf.get( - "spark.storage.replication.policy", classOf[RandomBlockReplicationPolicy].getName) + val priorityClass = conf.get(config.STORAGE_REPLICATION_POLICY) val clazz = Utils.classForName(priorityClass) val ret = clazz.getConstructor().newInstance().asInstanceOf[BlockReplicationPolicy] logInfo(s"Using $priorityClass for block replication policy") @@ -943,7 +942,7 @@ private[spark] class BlockManager( serializerInstance: SerializerInstance, bufferSize: Int, writeMetrics: ShuffleWriteMetricsReporter): DiskBlockObjectWriter = { - val syncWrites = conf.getBoolean("spark.shuffle.sync", false) + val syncWrites = conf.get(config.SHUFFLE_SYNC) new DiskBlockObjectWriter(file, serializerManager, serializerInstance, bufferSize, syncWrites, writeMetrics, blockId) } @@ -1339,7 +1338,7 @@ private[spark] class BlockManager( */ private def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { peerFetchLock.synchronized { - val cachedPeersTtl = conf.getInt("spark.storage.cachedPeersTtl", 60 * 1000) // milliseconds + val cachedPeersTtl = conf.get(config.STORAGE_CACHED_PEERS_TTL) // milliseconds val timeout = System.currentTimeMillis - lastPeerFetchTime > cachedPeersTtl if (cachedPeers == null || forceFetch || timeout) { cachedPeers = master.getPeers(blockManagerId).sortBy(_.hashCode) @@ -1393,7 +1392,7 @@ private[spark] class BlockManager( classTag: ClassTag[_], existingReplicas: Set[BlockManagerId] = Set.empty): Unit = { - val maxReplicationFailures = conf.getInt("spark.storage.maxReplicationFailures", 1) + val maxReplicationFailures = conf.get(config.STORAGE_MAX_REPLICATION_FAILURE) val tLevel = StorageLevel( useDisk = level.useDisk, useMemory = level.useMemory, 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 f984cf76e346..f5d6029e445c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -27,7 +27,7 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ @@ -60,7 +60,7 @@ class BlockManagerMasterEndpoint( private val topologyMapper = { val topologyMapperClassName = conf.get( - "spark.storage.replication.topologyMapper", classOf[DefaultTopologyMapper].getName) + config.STORAGE_REPLICATION_TOPOLOGY_MAPPER) val clazz = Utils.classForName(topologyMapperClassName) val mapper = clazz.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[TopologyMapper] @@ -68,7 +68,7 @@ class BlockManagerMasterEndpoint( mapper } - val proactivelyReplicate = conf.get("spark.storage.replication.proactive", "false").toBoolean + val proactivelyReplicate = conf.get(config.STORAGE_REPLICATION_PROACTIVE) logInfo("BlockManagerMasterEndpoint up") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 29963a95cb07..36cbaebfcc32 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -45,7 +45,7 @@ private[spark] class DiskStore( diskManager: DiskBlockManager, securityManager: SecurityManager) extends Logging { - private val minMemoryMapBytes = conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m") + private val minMemoryMapBytes = conf.get(config.STORAGE_MEMORY_MAP_THRESHOLD) private val maxMemoryMapBytes = conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS) private val blockSizes = new ConcurrentHashMap[BlockId, Long]() diff --git a/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala index a150a8e3636e..3c2c4b46dc4c 100644 --- a/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala +++ b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.Utils /** @@ -68,7 +68,7 @@ class DefaultTopologyMapper(conf: SparkConf) extends TopologyMapper(conf) with L */ @DeveloperApi class FileBasedTopologyMapper(conf: SparkConf) extends TopologyMapper(conf) with Logging { - val topologyFile = conf.getOption("spark.storage.replication.topologyFile") + val topologyFile = conf.get(config.STORAGE_REPLICATION_TOPOLOGY_FILE) require(topologyFile.isDefined, "Please specify topology file via " + "spark.storage.replication.topologyFile for FileBasedTopologyMapper.") val topologyMap = Utils.getPropertiesFromFile(topologyFile.get) 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 8513359934be..375d05be74bf 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 @@ -29,7 +29,7 @@ import com.google.common.io.ByteStreams import org.apache.spark.{SparkConf, TaskContext} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{UNROLL_MEMORY_CHECK_PERIOD, UNROLL_MEMORY_GROWTH_FACTOR} +import org.apache.spark.internal.config.{STORAGE_UNROLL_MEMORY_THRESHOLD, UNROLL_MEMORY_CHECK_PERIOD, UNROLL_MEMORY_GROWTH_FACTOR} import org.apache.spark.memory.{MemoryManager, MemoryMode} import org.apache.spark.serializer.{SerializationStream, SerializerManager} import org.apache.spark.storage._ @@ -100,7 +100,7 @@ private[spark] class MemoryStore( // Initial memory to request before unrolling any block private val unrollMemoryThreshold: Long = - conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + conf.get(STORAGE_UNROLL_MEMORY_THRESHOLD) /** Total amount of memory available for storage, in bytes. */ private def maxMemory: Long = { diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala index 3c4ee4eb6bbb..f0ae26e7a88e 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -21,6 +21,7 @@ import java.util.{Timer, TimerTask} import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.status.api.v1.StageData /** @@ -33,8 +34,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { // Carriage return private val CR = '\r' // Update period of progress bar, in milliseconds - private val updatePeriodMSec = - sc.getConf.getTimeAsMs("spark.ui.consoleProgress.update.interval", "200") + private val updatePeriodMSec = sc.getConf.get(UI_CONSOLE_PROGRESS_UPDATE_INTERVAL) // Delay to show up a progress bar, in milliseconds private val firstDelayMSec = 500L diff --git a/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala index da84fdf8fe14..fc9b50f14a08 100644 --- a/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala +++ b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala @@ -26,7 +26,7 @@ import scala.collection.JavaConverters._ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ /** * A servlet filter that implements HTTP security features. The following actions are taken 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 08f5fb937da7..e04eb8419b7d 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -40,7 +40,7 @@ import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{SecurityManager, SparkConf, SSLOptions} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.util.Utils /** @@ -500,7 +500,7 @@ private[spark] case class ServerInfo( * of the chain to perform security-related functions. */ private def addFilters(handler: ServletContextHandler, securityMgr: SecurityManager): Unit = { - conf.getOption("spark.ui.filters").toSeq.flatMap(Utils.stringToSeq).foreach { filter => + conf.get(UI_FILTERS).foreach { filter => logInfo(s"Adding filter to ${handler.getContextPath()}: $filter") val oldParams = conf.getOption(s"spark.$filter.params").toSeq .flatMap(Utils.stringToSeq) 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 d315ef66e0dc..7378801f7b4e 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.{JobExecutionStatus, SecurityManager, SparkConf, SparkContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1._ @@ -50,7 +51,7 @@ private[spark] class SparkUI private ( with Logging with UIRoot { - val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + val killEnabled = sc.map(_.conf.get(UI_KILL_ENABLED)).getOrElse(false) var appId: String = _ @@ -151,12 +152,11 @@ private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) } private[spark] object SparkUI { - val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" val DEFAULT_POOL_NAME = "default" def getUIPort(conf: SparkConf): Int = { - conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + conf.get(UI_PORT) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index 3d465a34e44a..cbb8b3c88c65 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -42,6 +42,8 @@ private[ui] class EnvironmentPage( propertyHeader, jvmRow, jvmInformation, fixedWidth = true) val sparkPropertiesTable = UIUtils.listingTable(propertyHeader, propertyRow, Utils.redact(conf, appEnv.sparkProperties.toSeq), fixedWidth = true) + val hadoopPropertiesTable = UIUtils.listingTable(propertyHeader, propertyRow, + Utils.redact(conf, appEnv.hadoopProperties.toSeq), fixedWidth = true) val systemPropertiesTable = UIUtils.listingTable( propertyHeader, propertyRow, appEnv.systemProperties, fixedWidth = true) val classpathEntriesTable = UIUtils.listingTable( @@ -70,26 +72,37 @@ private[ui] class EnvironmentPage(
{sparkPropertiesTable}
+ +

+ + Hadoop Properties +

+
+

- + System Properties

-
+

- + Classpath Entries

-
+ 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 d5a60f52cbb0..2ddd78accde8 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 @@ -21,6 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.apache.spark.internal.config.UI._ import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils, WebUIPage} private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { @@ -29,7 +30,7 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "exec private def init(): Unit = { val threadDumpEnabled = - parent.sc.isDefined && parent.conf.getBoolean("spark.ui.threadDumpsEnabled", true) + parent.sc.isDefined && parent.conf.get(UI_THREAD_DUMPS_ENABLED) attachPage(new ExecutorsPage(this, threadDumpEnabled)) if (threadDumpEnabled) { 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 8ec625da042f..2c94853f312e 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 @@ -27,6 +27,7 @@ import scala.xml.{Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.internal.config.UI._ import org.apache.spark.scheduler.TaskLocality import org.apache.spark.status._ import org.apache.spark.status.api.v1._ @@ -63,7 +64,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We // TODO: We should consider increasing the number of this parameter over time // if we find that it's okay. - private val MAX_TIMELINE_TASKS = parent.conf.getInt("spark.ui.timeline.tasks.maximum", 1000) + private val MAX_TIMELINE_TASKS = parent.conf.get(UI_TIMELINE_TASKS_MAXIMUM) private def getLocalitySummaryString(localitySummary: Map[String, Long]): String = { val names = Map( 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 348291fe5e7a..3370152975a5 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -171,11 +171,13 @@ private[spark] object JsonProtocol { val environmentDetails = environmentUpdate.environmentDetails val jvmInformation = mapToJson(environmentDetails("JVM Information").toMap) val sparkProperties = mapToJson(environmentDetails("Spark Properties").toMap) + val hadoopProperties = mapToJson(environmentDetails("Hadoop Properties").toMap) val systemProperties = mapToJson(environmentDetails("System Properties").toMap) val classpathEntries = mapToJson(environmentDetails("Classpath Entries").toMap) ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.environmentUpdate) ~ ("JVM Information" -> jvmInformation) ~ ("Spark Properties" -> sparkProperties) ~ + ("Hadoop Properties" -> hadoopProperties) ~ ("System Properties" -> systemProperties) ~ ("Classpath Entries" -> classpathEntries) } @@ -653,9 +655,13 @@ private[spark] object JsonProtocol { } def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { + // For compatible with previous event logs + val hadoopProperties = jsonOption(json \ "Hadoop Properties").map(mapFromJson(_).toSeq) + .getOrElse(Seq.empty) val environmentDetails = Map[String, Seq[(String, String)]]( "JVM Information" -> mapFromJson(json \ "JVM Information").toSeq, "Spark Properties" -> mapFromJson(json \ "Spark Properties").toSeq, + "Hadoop Properties" -> hadoopProperties, "System Properties" -> mapFromJson(json \ "System Properties").toSeq, "Classpath Entries" -> mapFromJson(json \ "Classpath Entries").toSeq) SparkListenerEnvironmentUpdate(environmentDetails) 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 16ef38142ad9..1a6dc1fa0cbd 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -61,6 +61,8 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} @@ -810,7 +812,7 @@ private[spark] object Utils extends Logging { } else { if (conf.getenv("MESOS_SANDBOX") != null && shuffleServiceEnabled) { logInfo("MESOS_SANDBOX available but not using provided Mesos sandbox because " + - "spark.shuffle.service.enabled is enabled.") + s"${config.SHUFFLE_SERVICE_ENABLED.key} is enabled.") } // In non-Yarn mode (or for the driver in yarn-client mode), we cannot trust the user // configuration to point to a secure directory. So create a subdirectory with restricted @@ -1456,16 +1458,12 @@ private[spark] object Utils extends Logging { CallSite(shortForm, longForm) } - private val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = - "spark.worker.ui.compressedLogFileLengthCacheSize" - private val DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE = 100 private var compressedLogFileLengthCache: LoadingCache[String, java.lang.Long] = null private def getCompressedLogFileLengthCache( sparkConf: SparkConf): LoadingCache[String, java.lang.Long] = this.synchronized { if (compressedLogFileLengthCache == null) { - val compressedLogFileLengthCacheSize = sparkConf.getInt( - UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF, - DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE) + val compressedLogFileLengthCacheSize = sparkConf.get( + UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF) compressedLogFileLengthCache = CacheBuilder.newBuilder() .maximumSize(compressedLogFileLengthCacheSize) .build[String, java.lang.Long](new CacheLoader[String, java.lang.Long]() { @@ -2387,8 +2385,7 @@ private[spark] object Utils extends Logging { // Returns the groups to which the current user belongs. def getCurrentUserGroups(sparkConf: SparkConf, username: String): Set[String] = { - val groupProviderClassName = sparkConf.get("spark.user.groups.mapping", - "org.apache.spark.security.ShellBasedGroupsMappingProvider") + val groupProviderClassName = sparkConf.get(USER_GROUPS_MAPPING) if (groupProviderClassName != "") { try { val groupMappingServiceProvider = classForName(groupProviderClassName). @@ -2535,8 +2532,7 @@ private[spark] object Utils extends Logging { * has its own mechanism to distribute jars. */ def getUserJars(conf: SparkConf): Seq[String] = { - val sparkJars = conf.getOption("spark.jars") - sparkJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten + conf.get(JARS).filter(_.nonEmpty) } /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 19ff109b673e..1ba3b7875f8d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -29,7 +29,7 @@ import com.google.common.io.ByteStreams import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer.{DeserializationStream, Serializer, SerializerManager} import org.apache.spark.storage.{BlockId, BlockManager} import org.apache.spark.util.CompletionIterator @@ -97,15 +97,14 @@ class ExternalAppendOnlyMap[K, V, C]( * NOTE: Setting this too low can cause excessive copying when serializing, since some serializers * grow internal data structures by growing + copying every time the number of objects doubles. */ - private val serializerBatchSize = sparkConf.getLong("spark.shuffle.spill.batchSize", 10000) + private val serializerBatchSize = sparkConf.get(config.SHUFFLE_SPILL_BATCH_SIZE) // Number of bytes spilled in total private var _diskBytesSpilled = 0L def diskBytesSpilled: Long = _diskBytesSpilled // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - private val fileBufferSize = - sparkConf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 + private val fileBufferSize = sparkConf.get(config.SHUFFLE_FILE_BUFFER_SIZE).toInt * 1024 // Write metrics private val writeMetrics: ShuffleWriteMetrics = new ShuffleWriteMetrics() diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 46279e79d78d..4806c1396725 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -27,7 +27,7 @@ import com.google.common.io.ByteStreams import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer._ import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter} @@ -109,7 +109,7 @@ private[spark] class ExternalSorter[K, V, C]( private val serInstance = serializer.newInstance() // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - private val fileBufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 + private val fileBufferSize = conf.get(config.SHUFFLE_FILE_BUFFER_SIZE).toInt * 1024 // Size of object batches when reading/writing from serializers. // @@ -118,7 +118,7 @@ private[spark] class ExternalSorter[K, V, C]( // // NOTE: Setting this too low can cause excessive copying when serializing, since some serializers // grow internal data structures by growing + copying every time the number of objects doubles. - private val serializerBatchSize = conf.getLong("spark.shuffle.spill.batchSize", 10000) + private val serializerBatchSize = conf.get(config.SHUFFLE_SPILL_BATCH_SIZE) // Data structures to store in-memory objects before we spill. Depending on whether we have an // Aggregator set, we either put objects into an AppendOnlyMap where we combine them, or we diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index 81457b53cd81..bfc0face5d8e 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -51,7 +51,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) // Initial threshold for the size of a collection before we start tracking its memory usage // For testing only private[this] val initialMemoryThreshold: Long = - SparkEnv.get.conf.getLong("spark.shuffle.spill.initialMemoryThreshold", 5 * 1024 * 1024) + SparkEnv.get.conf.get(SHUFFLE_SPILL_INITIAL_MEM_THRESHOLD) // Force this collection to spill when there are this many elements in memory // For testing only 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 aa5082f1ac7f..f34ae99c992d 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 @@ -45,6 +45,7 @@ import org.apache.spark.io.LZ4CompressionCodec; import org.apache.spark.io.LZFCompressionCodec; import org.apache.spark.io.SnappyCompressionCodec; +import org.apache.spark.internal.config.package$; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.network.util.LimitedInputStream; @@ -184,7 +185,7 @@ private List> readRecordsFromFile() throws IOException { fin.getChannel().position(startOffset); InputStream in = new LimitedInputStream(fin, partitionSize); in = blockManager.serializerManager().wrapForEncryption(in); - if (conf.getBoolean("spark.shuffle.compress", true)) { + if ((boolean) conf.get(package$.MODULE$.SHUFFLE_COMPRESS())) { in = CompressionCodec$.MODULE$.createCodec(conf).compressedInputStream(in); } try (DeserializationStream recordsStream = serializer.newInstance().deserializeStream(in)) { @@ -279,12 +280,12 @@ private void testMergingSpills( String compressionCodecName, boolean encrypt) throws Exception { if (compressionCodecName != null) { - conf.set("spark.shuffle.compress", "true"); + conf.set(package$.MODULE$.SHUFFLE_COMPRESS(), true); conf.set("spark.io.compression.codec", compressionCodecName); } else { - conf.set("spark.shuffle.compress", "false"); + conf.set(package$.MODULE$.SHUFFLE_COMPRESS(), false); } - conf.set(org.apache.spark.internal.config.package$.MODULE$.IO_ENCRYPTION_ENABLED(), encrypt); + conf.set(package$.MODULE$.IO_ENCRYPTION_ENABLED(), encrypt); SerializerManager manager; if (encrypt) { @@ -390,7 +391,7 @@ public void mergeSpillsWithFileStreamAndCompressionAndEncryption() throws Except @Test public void mergeSpillsWithCompressionAndEncryptionSlowPath() throws Exception { - conf.set("spark.shuffle.unsafe.fastMergeEnabled", "false"); + conf.set(package$.MODULE$.SHUFFLE_UNDAFE_FAST_MERGE_ENABLE(), false); testMergingSpills(false, LZ4CompressionCodec.class.getName(), true); } @@ -430,14 +431,14 @@ public void writeEnoughDataToTriggerSpill() throws Exception { @Test public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpillRadixOff() throws Exception { - conf.set("spark.shuffle.sort.useRadixSort", "false"); + conf.set(package$.MODULE$.SHUFFLE_SORT_USE_RADIXSORT(), false); writeEnoughRecordsToTriggerSortBufferExpansionAndSpill(); assertEquals(2, spillFilesCreated.size()); } @Test public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpillRadixOn() throws Exception { - conf.set("spark.shuffle.sort.useRadixSort", "true"); + conf.set(package$.MODULE$.SHUFFLE_SORT_USE_RADIXSORT(), true); writeEnoughRecordsToTriggerSortBufferExpansionAndSpill(); assertEquals(3, spillFilesCreated.size()); } 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 ecfebf8f8287..e4e0d47fc101 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 @@ -45,6 +45,7 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.util.Utils; +import org.apache.spark.internal.config.package$; import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; @@ -63,7 +64,7 @@ public abstract class AbstractBytesToBytesMapSuite { private TaskMemoryManager taskMemoryManager; private SerializerManager serializerManager = new SerializerManager( new JavaSerializer(new SparkConf()), - new SparkConf().set("spark.shuffle.spill.compress", "false")); + new SparkConf().set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false)); private static final long PAGE_SIZE_BYTES = 1L << 26; // 64 megabytes final LinkedList spillFilesCreated = new LinkedList<>(); @@ -79,8 +80,8 @@ public void setup() { new SparkConf() .set("spark.memory.offHeap.enabled", "" + useOffHeapMemoryAllocator()) .set("spark.memory.offHeap.size", "256mb") - .set("spark.shuffle.spill.compress", "false") - .set("spark.shuffle.compress", "false")); + .set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false) + .set(package$.MODULE$.SHUFFLE_COMPRESS(), false)); taskMemoryManager = new TaskMemoryManager(memoryManager, 0); tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "unsafe-test"); diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index d1b29d90ad91..a56743f738b6 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -63,7 +63,7 @@ public class UnsafeExternalSorterSuite { final TaskMemoryManager taskMemoryManager = new TaskMemoryManager(memoryManager, 0); final SerializerManager serializerManager = new SerializerManager( new JavaSerializer(conf), - conf.clone().set("spark.shuffle.spill.compress", "false")); + conf.clone().set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false)); // Use integer comparison for comparing prefixes (which are partition ids, in this case) final PrefixComparator prefixComparator = PrefixComparators.LONG; // Since the key fits within the 8-byte prefix, we don't need to do any record comparison, so diff --git a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json index 4ed053899ee6..a64617256d63 100644 --- a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json @@ -32,6 +32,11 @@ [ "spark.app.id", "app-20161116163331-0000" ], [ "spark.task.maxFailures", "4" ] ], + "hadoopProperties" : [ + [ "mapreduce.jobtracker.address", "local" ], + [ "yarn.resourcemanager.scheduler.monitor.policies", "org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy" ], + [ "mapreduce.jobhistory.client.thread-count", "10" ] + ], "systemProperties" : [ [ "java.io.tmpdir", "/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/" ], [ "line.separator", "\n" ], diff --git a/core/src/test/resources/spark-events/app-20161116163331-0000 b/core/src/test/resources/spark-events/app-20161116163331-0000 index 57cfc5b97312..8f77fdd34f43 100755 --- a/core/src/test/resources/spark-events/app-20161116163331-0000 +++ b/core/src/test/resources/spark-events/app-20161116163331-0000 @@ -1,6 +1,6 @@ {"Event":"SparkListenerLogStart","Spark Version":"2.1.0-SNAPSHOT"} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.167","Port":51475},"Maximum Memory":908381388,"Timestamp":1479335611477,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.167","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"51459","spark.repl.class.uri":"spark://172.22.0.167:51459/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"1000000","spark.app.id":"app-20161116163331-0000","spark.task.maxFailures":"4"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.167","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"51459","spark.repl.class.uri":"spark://172.22.0.167:51459/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"1000000","spark.app.id":"app-20161116163331-0000","spark.task.maxFailures":"4"},"Hadoop Properties":{"mapreduce.jobtracker.address":"local","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","mapreduce.jobhistory.client.thread-count":"10"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20161116163331-0000","Timestamp":1479335609916,"User":"jose"} {"Event":"SparkListenerExecutorAdded","Timestamp":1479335615320,"Executor ID":"3","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout","stderr":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.167","Port":51485},"Maximum Memory":908381388,"Timestamp":1479335615387,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 6d9e47cfd00f..0e019e473a88 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import com.google.common.io.ByteStreams import org.apache.hadoop.fs.Path +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.rdd._ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} @@ -589,7 +590,7 @@ class CheckpointCompressionSuite extends SparkFunSuite with LocalSparkContext { withTempDir { checkpointDir => val conf = new SparkConf() .set("spark.checkpoint.compress", "true") - .set("spark.ui.enabled", "false") + .set(UI_ENABLED.key, "false") sc = new SparkContext("local", "test", conf) sc.setCheckpointDir(checkpointDir.toString) val rdd = sc.makeRDD(1 to 20, numSlices = 1) diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 1fcc975ab39a..9e282844f9fe 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration import org.scalatest.time.SpanSugar._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage._ @@ -49,7 +49,7 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[So .set("spark.cleaner.referenceTracking.blocking", "true") .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.cleaner.referenceTracking.cleanCheckpoints", "true") - .set("spark.shuffle.manager", shuffleManager.getName) + .set(config.SHUFFLE_MANAGER, shuffleManager.getName) before { sc = new SparkContext(conf) @@ -319,7 +319,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") - .set("spark.shuffle.manager", shuffleManager.getName) + .set(config.SHUFFLE_MANAGER, shuffleManager.getName) sc = new SparkContext(conf2) val numRdds = 10 diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 21050e44414f..17ffc7bd36c6 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -87,7 +87,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex } test("groupByKey where map output sizes exceed maxMbInFlight") { - val conf = new SparkConf().set("spark.reducer.maxSizeInFlight", "1m") + val conf = new SparkConf().set(config.REDUCER_MAX_SIZE_IN_FLIGHT.key, "1m") sc = new SparkContext(clusterUrl, "test", conf) // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output // file should be about 2.5 MB @@ -217,8 +217,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("compute without caching when no partitions fit in memory") { val size = 10000 val conf = new SparkConf() - .set("spark.storage.unrollMemoryThreshold", "1024") + .set(config.STORAGE_UNROLL_MEMORY_THRESHOLD, 1024L) .set(TEST_MEMORY, size.toLong / 2) + sc = new SparkContext(clusterUrl, "test", conf) val data = sc.parallelize(1 to size, 2).persist(StorageLevel.MEMORY_ONLY) assert(data.count() === size) @@ -233,8 +234,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex val size = 10000 val numPartitions = 20 val conf = new SparkConf() - .set("spark.storage.unrollMemoryThreshold", "1024") + .set(config.STORAGE_UNROLL_MEMORY_THRESHOLD, 1024L) .set(TEST_MEMORY, size.toLong) + sc = new SparkContext(clusterUrl, "test", conf) val data = sc.parallelize(1 to size, numPartitions).persist(StorageLevel.MEMORY_ONLY) assert(data.count() === size) diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 462d5f5604ae..262e2a7ef5c9 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -42,9 +42,9 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { val transportContext = new TransportContext(transportConf, rpcHandler) server = transportContext.createServer() - conf.set("spark.shuffle.manager", "sort") - conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") - conf.set(config.SHUFFLE_SERVICE_PORT.key, server.getPort.toString) + conf.set(config.SHUFFLE_MANAGER, "sort") + conf.set(config.SHUFFLE_SERVICE_ENABLED, true) + conf.set(config.SHUFFLE_SERVICE_PORT, server.getPort) } override def afterAll() { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index f2d97d452ddb..5f79b526a419 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.{IOException, NotSerializableException, ObjectInputStream} +import org.apache.spark.internal.config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK import org.apache.spark.memory.TestMemoryConsumer import org.apache.spark.storage.StorageLevel import org.apache.spark.util.NonSerializable @@ -144,7 +145,7 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { } test("managed memory leak error should not mask other failures (SPARK-9266") { - val conf = new SparkConf().set("spark.unsafe.exceptionOnMemoryLeak", "true") + val conf = new SparkConf().set(UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) sc = new SparkContext("local[1,1]", "test", conf) // If a task leaks memory but fails due to some other cause, then make sure that the original diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 61da4138896c..f8adaf59fa0a 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -27,6 +27,7 @@ import scala.concurrent.duration._ import org.scalatest.BeforeAndAfter import org.scalatest.Matchers +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted, SparkListenerTaskEnd, SparkListenerTaskStart} import org.apache.spark.util.ThreadUtils @@ -256,7 +257,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft .set("spark.task.reaper.enabled", "true") .set("spark.task.reaper.killTimeout", "-1") .set("spark.task.reaper.PollingInterval", "1s") - .set("spark.deploy.maxExecutorRetries", "1") + .set(MAX_EXECUTOR_RETRIES, 1) sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) // Add a listener to release the semaphore once any tasks are launched. diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index c088da8fbf3b..adaa069ff227 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -24,6 +24,7 @@ import org.mockito.Mockito._ import org.apache.spark.LocalSparkContext._ import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.internal.config._ import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEnv} import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException @@ -171,7 +172,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { val newConf = new SparkConf newConf.set("spark.rpc.message.maxSize", "1") newConf.set("spark.rpc.askTimeout", "1") // Fail fast - newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "1048576") + newConf.set(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST, 1048576L) val masterTracker = newTrackerMaster(newConf) val rpcEnv = createRpcEnv("spark") @@ -200,7 +201,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { val newConf = new SparkConf newConf.set("spark.rpc.message.maxSize", "1") newConf.set("spark.rpc.askTimeout", "1") // Fail fast - newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", Int.MaxValue.toString) + newConf.set(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST, Int.MaxValue.toLong) intercept[IllegalArgumentException] { newTrackerMaster(newConf) } } @@ -244,7 +245,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { val newConf = new SparkConf newConf.set("spark.rpc.message.maxSize", "1") newConf.set("spark.rpc.askTimeout", "1") // Fail fast - newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "10240") // 10 KiB << 1MiB framesize + newConf.set(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST, 10240L) // 10 KiB << 1MiB framesize // needs TorrentBroadcast so need a SparkContext withSpark(new SparkContext("local", "MapOutputTrackerSuite", newConf)) { sc => diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index e9061f4e7beb..9f0d2ac82ead 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -26,6 +26,7 @@ import java.util.Base64 import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils} @@ -43,11 +44,11 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with conf") { val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.ui.view.acls", "user1,user2") - val securityManager = new SecurityManager(conf); + conf.set(NETWORK_AUTH_ENABLED, true) + conf.set(AUTH_SECRET, "good") + conf.set(ACLS_ENABLE, true) + conf.set(UI_VIEW_ACLS, Seq("user1", "user2")) + val securityManager = new SecurityManager(conf) assert(securityManager.isAuthenticationEnabled() === true) assert(securityManager.aclsEnabled() === true) assert(securityManager.checkUIViewPermissions("user1") === true) @@ -57,10 +58,10 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with conf for groups") { val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.ui.view.acls.groups", "group1,group2") + conf.set(NETWORK_AUTH_ENABLED, true) + conf.set(AUTH_SECRET, "good") + conf.set(ACLS_ENABLE, true) + conf.set(UI_VIEW_ACLS_GROUPS, Seq("group1", "group2")) // default ShellBasedGroupsMappingProvider is used to resolve user groups val securityManager = new SecurityManager(conf); // assuming executing user does not belong to group1,group2 @@ -68,27 +69,27 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user2") === false) val conf2 = new SparkConf - conf2.set("spark.authenticate", "true") - conf2.set("spark.authenticate.secret", "good") - conf2.set("spark.ui.acls.enable", "true") - conf2.set("spark.ui.view.acls.groups", "group1,group2") + conf2.set(NETWORK_AUTH_ENABLED, true) + conf2.set(AUTH_SECRET, "good") + conf2.set(ACLS_ENABLE, true) + conf2.set(UI_VIEW_ACLS_GROUPS, Seq("group1", "group2")) // explicitly specify a custom GroupsMappingServiceProvider - conf2.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf2.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager2 = new SecurityManager(conf2); + val securityManager2 = new SecurityManager(conf2) // group4,group5 do not match assert(securityManager2.checkUIViewPermissions("user1") === true) assert(securityManager2.checkUIViewPermissions("user2") === true) val conf3 = new SparkConf - conf3.set("spark.authenticate", "true") - conf3.set("spark.authenticate.secret", "good") - conf3.set("spark.ui.acls.enable", "true") - conf3.set("spark.ui.view.acls.groups", "group4,group5") + conf3.set(NETWORK_AUTH_ENABLED, true) + conf3.set(AUTH_SECRET, "good") + conf3.set(ACLS_ENABLE, true) + conf3.set(UI_VIEW_ACLS_GROUPS, Seq("group4", "group5")) // explicitly specify a bogus GroupsMappingServiceProvider - conf3.set("spark.user.groups.mapping", "BogusServiceProvider") + conf3.set(USER_GROUPS_MAPPING, "BogusServiceProvider") - val securityManager3 = new SecurityManager(conf3); + val securityManager3 = new SecurityManager(conf3) // BogusServiceProvider cannot be loaded and an error is logged returning an empty group set assert(securityManager3.checkUIViewPermissions("user1") === false) assert(securityManager3.checkUIViewPermissions("user2") === false) @@ -96,7 +97,7 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with api") { val conf = new SparkConf - conf.set("spark.ui.view.acls", "user1,user2") + conf.set(UI_VIEW_ACLS, Seq("user1", "user2")) val securityManager = new SecurityManager(conf); securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) @@ -108,7 +109,7 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) - securityManager.setViewAcls(Set[String]("user5"), "user6,user7") + securityManager.setViewAcls(Set[String]("user5"), Seq("user6", "user7")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user5") === true) assert(securityManager.checkUIViewPermissions("user6") === true) @@ -119,41 +120,41 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with api for groups") { val conf = new SparkConf - conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) - securityManager.setViewAclsGroups("group1,group2") + securityManager.setViewAclsGroups(Seq("group1", "group2")) // group1,group2 match assert(securityManager.checkUIViewPermissions("user1") === true) assert(securityManager.checkUIViewPermissions("user2") === true) // change groups so they do not match - securityManager.setViewAclsGroups("group4,group5") + securityManager.setViewAclsGroups(Seq("group4", "group5")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) val conf2 = new SparkConf - conf.set("spark.user.groups.mapping", "BogusServiceProvider") + conf.set(USER_GROUPS_MAPPING, "BogusServiceProvider") val securityManager2 = new SecurityManager(conf2) securityManager2.setAcls(true) - securityManager2.setViewAclsGroups("group1,group2") + securityManager2.setViewAclsGroups(Seq("group1", "group2")) // group1,group2 do not match because of BogusServiceProvider assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) // setting viewAclsGroups to empty should still not match because of BogusServiceProvider - securityManager2.setViewAclsGroups("") + securityManager2.setViewAclsGroups(Nil) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) } test("set security modify acls") { val conf = new SparkConf - conf.set("spark.modify.acls", "user1,user2") + conf.set(MODIFY_ACLS, Seq("user1", "user2")) val securityManager = new SecurityManager(conf); securityManager.setAcls(true) @@ -166,7 +167,7 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) - securityManager.setModifyAcls(Set("user5"), "user6,user7") + securityManager.setModifyAcls(Set("user5"), Seq("user6", "user7")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user5") === true) assert(securityManager.checkModifyPermissions("user6") === true) @@ -177,34 +178,35 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security modify acls for groups") { val conf = new SparkConf - conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) - securityManager.setModifyAclsGroups("group1,group2") + securityManager.setModifyAclsGroups(Seq("group1", "group2")) // group1,group2 match assert(securityManager.checkModifyPermissions("user1") === true) assert(securityManager.checkModifyPermissions("user2") === true) // change groups so they do not match - securityManager.setModifyAclsGroups("group4,group5") + securityManager.setModifyAclsGroups(Seq("group4", "group5")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user2") === false) // change so they match again - securityManager.setModifyAclsGroups("group2,group3") + securityManager.setModifyAclsGroups(Seq("group2", "group3")) + assert(securityManager.checkModifyPermissions("user1") === true) assert(securityManager.checkModifyPermissions("user2") === true) } test("set security admin acls") { val conf = new SparkConf - conf.set("spark.admin.acls", "user1,user2") - conf.set("spark.ui.view.acls", "user3") - conf.set("spark.modify.acls", "user4") + conf.set(ADMIN_ACLS, Seq("user1", "user2")) + conf.set(UI_VIEW_ACLS, Seq("user3")) + conf.set(MODIFY_ACLS, Seq("user4")) - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) @@ -221,9 +223,9 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user5") === false) assert(securityManager.checkUIViewPermissions(null) === true) - securityManager.setAdminAcls("user6") - securityManager.setViewAcls(Set[String]("user8"), "user9") - securityManager.setModifyAcls(Set("user11"), "user9") + securityManager.setAdminAcls(Seq("user6")) + securityManager.setViewAcls(Set[String]("user8"), Seq("user9")) + securityManager.setModifyAcls(Set("user11"), Seq("user9")) assert(securityManager.checkModifyPermissions("user6") === true) assert(securityManager.checkModifyPermissions("user11") === true) assert(securityManager.checkModifyPermissions("user9") === true) @@ -240,12 +242,12 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security admin acls for groups") { val conf = new SparkConf - conf.set("spark.admin.acls.groups", "group1") - conf.set("spark.ui.view.acls.groups", "group2") - conf.set("spark.modify.acls.groups", "group3") - conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf.set(ADMIN_ACLS_GROUPS, Seq("group1")) + conf.set(UI_VIEW_ACLS_GROUPS, Seq("group2")) + conf.set(MODIFY_ACLS_GROUPS, Seq("group3")) + conf.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) assert(securityManager.aclsEnabled() === true) @@ -254,38 +256,38 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user1") === true) // change admin groups so they do not match. view and modify groups are set to admin groups - securityManager.setAdminAclsGroups("group4,group5") + securityManager.setAdminAclsGroups(Seq("group4", "group5")) // invoke the set ui and modify to propagate the changes - securityManager.setViewAclsGroups("") - securityManager.setModifyAclsGroups("") + securityManager.setViewAclsGroups(Nil) + securityManager.setModifyAclsGroups(Nil) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user1") === false) // change modify groups so they match - securityManager.setModifyAclsGroups("group3") + securityManager.setModifyAclsGroups(Seq("group3")) assert(securityManager.checkModifyPermissions("user1") === true) assert(securityManager.checkUIViewPermissions("user1") === false) // change view groups so they match - securityManager.setViewAclsGroups("group2") - securityManager.setModifyAclsGroups("group4") + securityManager.setViewAclsGroups(Seq("group2")) + securityManager.setModifyAclsGroups(Seq("group4")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user1") === true) // change modify and view groups so they do not match - securityManager.setViewAclsGroups("group7") - securityManager.setModifyAclsGroups("group8") + securityManager.setViewAclsGroups(Seq("group7")) + securityManager.setModifyAclsGroups(Seq("group8")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user1") === false) } test("set security with * in acls") { val conf = new SparkConf - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.admin.acls", "user1,user2") - conf.set("spark.ui.view.acls", "*") - conf.set("spark.modify.acls", "user4") + conf.set(ACLS_ENABLE.key, "true") + conf.set(ADMIN_ACLS, Seq("user1", "user2")) + conf.set(UI_VIEW_ACLS, Seq("*")) + conf.set(MODIFY_ACLS, Seq("user4")) val securityManager = new SecurityManager(conf) assert(securityManager.aclsEnabled() === true) @@ -299,22 +301,22 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user8") === false) // check for modifyAcls with * - securityManager.setModifyAcls(Set("user4"), "*") + securityManager.setModifyAcls(Set("user4"), Seq("*")) assert(securityManager.checkModifyPermissions("user7") === true) assert(securityManager.checkModifyPermissions("user8") === true) - securityManager.setAdminAcls("user1,user2") - securityManager.setModifyAcls(Set("user1"), "user2") - securityManager.setViewAcls(Set("user1"), "user2") + securityManager.setAdminAcls(Seq("user1", "user2")) + securityManager.setModifyAcls(Set("user1"), Seq("user2")) + securityManager.setViewAcls(Set("user1"), Seq("user2")) assert(securityManager.checkUIViewPermissions("user5") === false) assert(securityManager.checkUIViewPermissions("user6") === false) assert(securityManager.checkModifyPermissions("user7") === false) assert(securityManager.checkModifyPermissions("user8") === false) // check for adminAcls with * - securityManager.setAdminAcls("user1,*") - securityManager.setModifyAcls(Set("user1"), "user2") - securityManager.setViewAcls(Set("user1"), "user2") + securityManager.setAdminAcls(Seq("user1", "*")) + securityManager.setModifyAcls(Set("user1"), Seq("user2")) + securityManager.setViewAcls(Set("user1"), Seq("user2")) assert(securityManager.checkUIViewPermissions("user5") === true) assert(securityManager.checkUIViewPermissions("user6") === true) assert(securityManager.checkModifyPermissions("user7") === true) @@ -323,10 +325,10 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with * in acls for groups") { val conf = new SparkConf - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.admin.acls.groups", "group4,group5") - conf.set("spark.ui.view.acls.groups", "*") - conf.set("spark.modify.acls.groups", "group6") + conf.set(ACLS_ENABLE, true) + conf.set(ADMIN_ACLS_GROUPS, Seq("group4", "group5")) + conf.set(UI_VIEW_ACLS_GROUPS, Seq("*")) + conf.set(MODIFY_ACLS_GROUPS, Seq("group6")) val securityManager = new SecurityManager(conf) assert(securityManager.aclsEnabled() === true) @@ -338,17 +340,17 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user2") === false) // check for modifyAcls with * - securityManager.setModifyAclsGroups("*") - securityManager.setViewAclsGroups("group6") + securityManager.setModifyAclsGroups(Seq("*")) + securityManager.setViewAclsGroups(Seq("group6")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) assert(securityManager.checkModifyPermissions("user1") === true) assert(securityManager.checkModifyPermissions("user2") === true) // check for adminAcls with * - securityManager.setAdminAclsGroups("group9,*") - securityManager.setModifyAclsGroups("group4,group5") - securityManager.setViewAclsGroups("group6,group7") + securityManager.setAdminAclsGroups(Seq("group9", "*")) + securityManager.setModifyAclsGroups(Seq("group4", "group5")) + securityManager.setViewAclsGroups(Seq("group6", "group7")) assert(securityManager.checkUIViewPermissions("user5") === true) assert(securityManager.checkUIViewPermissions("user6") === true) assert(securityManager.checkModifyPermissions("user7") === true) @@ -367,13 +369,13 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user1") === false) // set groups only - securityManager.setAdminAclsGroups("group1,group2") + securityManager.setAdminAclsGroups(Seq("group1", "group2")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user1") === false) } test("missing secret authentication key") { - val conf = new SparkConf().set("spark.authenticate", "true") + val conf = new SparkConf().set(NETWORK_AUTH_ENABLED, true) val mgr = new SecurityManager(conf) intercept[IllegalArgumentException] { mgr.getSecretKey() diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index ffa70425ea36..8b1084a8edc7 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -23,6 +23,7 @@ import java.util.concurrent.{Callable, CyclicBarrier, Executors, ExecutorService import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass +import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} @@ -41,7 +42,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC conf.set(TEST_NO_STAGE_RETRY, true) test("groupByKey without compression") { - val myConf = conf.clone().set("spark.shuffle.compress", "false") + val myConf = conf.clone().set(config.SHUFFLE_COMPRESS, false) sc = new SparkContext("local", "test", myConf) val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) val groups = pairs.groupByKey(4).collect() @@ -215,7 +216,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC test("sort with Java non serializable class - Kryo") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - val myConf = conf.clone().set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + val myConf = conf.clone().set(config.SERIALIZER, "org.apache.spark.serializer.KryoSerializer") sc = new SparkContext("local-cluster[2,1,1024]", "test", myConf) val a = sc.parallelize(1 to 10, 2) val b = a.map { x => @@ -251,8 +252,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val myConf = conf.clone() .setAppName("test") .setMaster("local") - .set("spark.shuffle.spill.compress", shuffleSpillCompress.toString) - .set("spark.shuffle.compress", shuffleCompress.toString) + .set(config.SHUFFLE_SPILL_COMPRESS, shuffleSpillCompress) + .set(config.SHUFFLE_COMPRESS, shuffleCompress) resetSparkContext() sc = new SparkContext(myConf) val diskBlockManager = sc.env.blockManager.diskBlockManager @@ -262,8 +263,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(diskBlockManager.getAllFiles().nonEmpty) } catch { case e: Exception => - val errMsg = s"Failed with spark.shuffle.spill.compress=$shuffleSpillCompress," + - s" spark.shuffle.compress=$shuffleCompress" + val errMsg = s"Failed with ${config.SHUFFLE_SPILL_COMPRESS.key}=$shuffleSpillCompress," + + s" ${config.SHUFFLE_COMPRESS.key}=$shuffleCompress" throw new Exception(errMsg, e) } } diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index c0126e41ff7f..1aceda498d7c 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -25,6 +25,7 @@ import org.apache.commons.io.FileUtils import org.apache.commons.io.filefilter.TrueFileFilter import org.scalatest.BeforeAndAfterAll +import org.apache.spark.internal.config import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.shuffle.sort.SortShuffleManager @@ -42,7 +43,7 @@ class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { // before/after a test, it could return the same directory even if this property // is configured. Utils.clearLocalRootDirs() - conf.set("spark.shuffle.manager", "sort") + conf.set(config.SHUFFLE_MANAGER, "sort") } override def beforeEach(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 9a6abbdb0a46..4071dd4eff07 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -28,6 +28,7 @@ import com.esotericsoftware.kryo.Kryo import org.apache.spark.internal.config._ import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.serializer.{JavaSerializer, KryoRegistrator, KryoSerializer} import org.apache.spark.util.{ResetSystemProperties, RpcUtils} @@ -78,7 +79,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(conf.get("spark.master") === "local[3]") assert(conf.get("spark.app.name") === "My app") assert(conf.get("spark.home") === "/path") - assert(conf.get("spark.jars") === "a.jar,b.jar") + assert(conf.get(JARS) === Seq("a.jar", "b.jar")) assert(conf.get("spark.executorEnv.VAR1") === "value1") assert(conf.get("spark.executorEnv.VAR2") === "value2") assert(conf.get("spark.executorEnv.VAR3") === "value3") @@ -86,7 +87,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // Test the Java-friendly versions of these too conf.setJars(Array("c.jar", "d.jar")) conf.setExecutorEnv(Array(("VAR4", "value4"), ("VAR5", "value5"))) - assert(conf.get("spark.jars") === "c.jar,d.jar") + assert(conf.get(JARS) === Seq("c.jar", "d.jar")) assert(conf.get("spark.executorEnv.VAR4") === "value4") assert(conf.get("spark.executorEnv.VAR5") === "value5") } @@ -182,19 +183,19 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("register kryo classes through registerKryoClasses") { - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) conf.registerKryoClasses(Array(classOf[Class1], classOf[Class2])) - assert(conf.get("spark.kryo.classesToRegister") === - classOf[Class1].getName + "," + classOf[Class2].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === + Seq(classOf[Class1].getName, classOf[Class2].getName).toSet) conf.registerKryoClasses(Array(classOf[Class3])) - assert(conf.get("spark.kryo.classesToRegister") === - classOf[Class1].getName + "," + classOf[Class2].getName + "," + classOf[Class3].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === + Seq(classOf[Class1].getName, classOf[Class2].getName, classOf[Class3].getName).toSet) conf.registerKryoClasses(Array(classOf[Class2])) - assert(conf.get("spark.kryo.classesToRegister") === - classOf[Class1].getName + "," + classOf[Class2].getName + "," + classOf[Class3].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === + Seq(classOf[Class1].getName, classOf[Class2].getName, classOf[Class3].getName).toSet) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. @@ -205,12 +206,12 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("register kryo classes through registerKryoClasses and custom registrator") { - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) conf.registerKryoClasses(Array(classOf[Class1])) - assert(conf.get("spark.kryo.classesToRegister") === classOf[Class1].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === Seq(classOf[Class1].getName).toSet) - conf.set("spark.kryo.registrator", classOf[CustomRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, classOf[CustomRegistrator].getName) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. @@ -220,9 +221,9 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("register kryo classes through conf") { - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") - conf.set("spark.kryo.classesToRegister", "java.lang.StringBuffer") - conf.set("spark.serializer", classOf[KryoSerializer].getName) + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) + conf.set(KRYO_CLASSES_TO_REGISTER, Seq("java.lang.StringBuffer")) + conf.set(SERIALIZER, classOf[KryoSerializer].getName) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. @@ -290,12 +291,12 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // set the conf in the deprecated way conf.set("spark.io.compression.lz4.block.size", "12345") // get the conf in the recommended way - assert(conf.get("spark.io.compression.lz4.blockSize") === "12345") + assert(conf.get(IO_COMPRESSION_LZ4_BLOCKSIZE.key) === "12345") // we can still get the conf in the deprecated way assert(conf.get("spark.io.compression.lz4.block.size") === "12345") // the contains() also works as expected assert(conf.contains("spark.io.compression.lz4.block.size")) - assert(conf.contains("spark.io.compression.lz4.blockSize")) + assert(conf.contains(IO_COMPRESSION_LZ4_BLOCKSIZE.key)) assert(conf.contains("spark.io.unknown") === false) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index f8938dfedee5..811b9757232e 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -23,110 +23,129 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.scheduler.local.LocalSchedulerBackend +import org.apache.spark.util.Utils class SparkContextSchedulerCreationSuite extends SparkFunSuite with LocalSparkContext with PrivateMethodTester with Logging { - def createTaskScheduler(master: String): TaskSchedulerImpl = - createTaskScheduler(master, "client") + def noOp(taskSchedulerImpl: TaskSchedulerImpl): Unit = {} - def createTaskScheduler(master: String, deployMode: String): TaskSchedulerImpl = - createTaskScheduler(master, deployMode, new SparkConf()) + def createTaskScheduler(master: String)(body: TaskSchedulerImpl => Unit = noOp): Unit = + createTaskScheduler(master, "client")(body) + + def createTaskScheduler(master: String, deployMode: String)( + body: TaskSchedulerImpl => Unit): Unit = + createTaskScheduler(master, deployMode, new SparkConf())(body) def createTaskScheduler( master: String, deployMode: String, - conf: SparkConf): TaskSchedulerImpl = { + conf: SparkConf)(body: TaskSchedulerImpl => Unit): Unit = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. sc = new SparkContext("local", "test", conf) val createTaskSchedulerMethod = PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) - val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, deployMode) - sched.asInstanceOf[TaskSchedulerImpl] + val (_, sched) = + SparkContext invokePrivate createTaskSchedulerMethod(sc, master, deployMode) + try { + body(sched.asInstanceOf[TaskSchedulerImpl]) + } finally { + Utils.tryLogNonFatalError { + sched.stop() + } + } } test("bad-master") { val e = intercept[SparkException] { - createTaskScheduler("localhost:1234") + createTaskScheduler("localhost:1234")() } assert(e.getMessage.contains("Could not parse Master URL")) } test("local") { - val sched = createTaskScheduler("local") - sched.backend match { - case s: LocalSchedulerBackend => assert(s.totalCores === 1) - case _ => fail() + val sched = createTaskScheduler("local") { sched => + sched.backend match { + case s: LocalSchedulerBackend => assert(s.totalCores === 1) + case _ => fail() + } } } test("local-*") { - val sched = createTaskScheduler("local[*]") - sched.backend match { - case s: LocalSchedulerBackend => - assert(s.totalCores === Runtime.getRuntime.availableProcessors()) - case _ => fail() + val sched = createTaskScheduler("local[*]") { sched => + sched.backend match { + case s: LocalSchedulerBackend => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } } } test("local-n") { - val sched = createTaskScheduler("local[5]") - assert(sched.maxTaskFailures === 1) - sched.backend match { - case s: LocalSchedulerBackend => assert(s.totalCores === 5) - case _ => fail() + val sched = createTaskScheduler("local[5]") { sched => + assert(sched.maxTaskFailures === 1) + sched.backend match { + case s: LocalSchedulerBackend => assert(s.totalCores === 5) + case _ => fail() + } } } test("local-*-n-failures") { - val sched = createTaskScheduler("local[* ,2]") - assert(sched.maxTaskFailures === 2) - sched.backend match { - case s: LocalSchedulerBackend => - assert(s.totalCores === Runtime.getRuntime.availableProcessors()) - case _ => fail() + val sched = createTaskScheduler("local[* ,2]") { sched => + assert(sched.maxTaskFailures === 2) + sched.backend match { + case s: LocalSchedulerBackend => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } } } test("local-n-failures") { - val sched = createTaskScheduler("local[4, 2]") - assert(sched.maxTaskFailures === 2) - sched.backend match { - case s: LocalSchedulerBackend => assert(s.totalCores === 4) - case _ => fail() + val sched = createTaskScheduler("local[4, 2]") { sched => + assert(sched.maxTaskFailures === 2) + sched.backend match { + case s: LocalSchedulerBackend => assert(s.totalCores === 4) + case _ => fail() + } } } test("bad-local-n") { val e = intercept[SparkException] { - createTaskScheduler("local[2*]") + createTaskScheduler("local[2*]")() } assert(e.getMessage.contains("Could not parse Master URL")) } test("bad-local-n-failures") { val e = intercept[SparkException] { - createTaskScheduler("local[2*,4]") + createTaskScheduler("local[2*,4]")() } assert(e.getMessage.contains("Could not parse Master URL")) } test("local-default-parallelism") { val conf = new SparkConf().set("spark.default.parallelism", "16") - val sched = createTaskScheduler("local", "client", conf) - sched.backend match { - case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16) - case _ => fail() + val sched = createTaskScheduler("local", "client", conf) { sched => + sched.backend match { + case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16) + case _ => fail() + } } } test("local-cluster") { - createTaskScheduler("local-cluster[3, 14, 1024]").backend match { - case s: StandaloneSchedulerBackend => // OK - case _ => fail() + createTaskScheduler("local-cluster[3, 14, 1024]") { sched => + sched.backend match { + case s: StandaloneSchedulerBackend => // OK + case _ => fail() + } } } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 66de2f2ac86a..41d5dee4189f 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -33,7 +33,8 @@ import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFor import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually -import org.apache.spark.internal.config.EXECUTOR_HEARTBEAT_DROP_ZERO_ACCUMULATOR_UPDATES +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorMetricsUpdate, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util.{ThreadUtils, Utils} @@ -665,7 +666,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu val conf = new SparkConf() .setMaster("local-cluster[1,2,1024]") .setAppName("test-cluster") - .set("spark.ui.enabled", "false") + .set(UI_ENABLED.key, "false") // Disable this so that if a task is running, we can make sure the executor will always send // task metrics via heartbeat to driver. .set(EXECUTOR_HEARTBEAT_DROP_ZERO_ACCUMULATOR_UPDATES.key, "false") diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala index 7407a656dbfc..24004de75105 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala @@ -24,6 +24,7 @@ import scala.io.Source import org.scalatest.Matchers import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils @@ -48,7 +49,7 @@ class PythonBroadcastSuite extends SparkFunSuite with Matchers with SharedSparkC } val broadcast = new PythonBroadcast(broadcastDataFile.getAbsolutePath) assertBroadcastIsValid(broadcast) - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) val deserializedBroadcast = Utils.clone[PythonBroadcast](broadcast, new KryoSerializer(conf).newInstance()) assertBroadcastIsValid(deserializedBroadcast) diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 6976464e8ab5..18ec60dd9e01 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -24,6 +24,8 @@ import scala.util.Random import org.scalatest.Assertions import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.internal.config.SERIALIZER import org.apache.spark.io.SnappyCompressionCodec import org.apache.spark.rdd.RDD import org.apache.spark.security.EncryptionFunSuite @@ -67,8 +69,8 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio encryptionTest("Accessing TorrentBroadcast variables in a local cluster") { conf => val numSlaves = 4 - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.broadcast.compress", "true") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(config.BROADCAST_COMPRESS, true) sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test", conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) @@ -145,7 +147,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio encryptionTest("Cache broadcast to disk") { conf => conf.setMaster("local") .setAppName("test") - .set("spark.memory.storageFraction", "0.0") + .set(config.MEMORY_STORAGE_FRACTION, 0.0) sc = new SparkContext(conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) @@ -172,7 +174,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio val conf = new SparkConf() .setMaster("local[4]") .setAppName("test") - .set("spark.memory.storageFraction", "0.0") + .set(config.MEMORY_STORAGE_FRACTION, 0.0) sc = new SparkContext(conf) val list = List[Int](1, 2, 3, 4) 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 2a7a55cbb903..30efbb000d80 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -42,6 +42,7 @@ import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.scheduler.EventLoggingListener import org.apache.spark.util.{CommandLineUtils, ResetSystemProperties, Utils} @@ -220,7 +221,7 @@ class SparkSubmitSuite val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) appArgs.deployMode should be ("client") - conf.get("spark.submit.deployMode") should be ("client") + conf.get(SUBMIT_DEPLOY_MODE) should be ("client") // Both cmd line and configuration are specified, cmdline option takes the priority val clArgs1 = Seq( @@ -234,7 +235,7 @@ class SparkSubmitSuite val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1) appArgs1.deployMode should be ("cluster") - conf1.get("spark.submit.deployMode") should be ("cluster") + conf1.get(SUBMIT_DEPLOY_MODE) should be ("cluster") // Neither cmdline nor configuration are specified, client mode is the default choice val clArgs2 = Seq( @@ -247,7 +248,7 @@ class SparkSubmitSuite val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) appArgs2.deployMode should be ("client") - conf2.get("spark.submit.deployMode") should be ("client") + conf2.get(SUBMIT_DEPLOY_MODE) should be ("client") } test("handles YARN cluster mode") { @@ -289,7 +290,7 @@ class SparkSubmitSuite conf.get("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt") conf.get("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") conf.get("spark.app.name") should be ("beauty") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) sys.props("SPARK_SUBMIT") should be ("true") } @@ -328,7 +329,7 @@ class SparkSubmitSuite conf.get("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") conf.get("spark.yarn.dist.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) sys.props("SPARK_SUBMIT") should be ("true") } @@ -373,13 +374,13 @@ class SparkSubmitSuite val confMap = conf.getAll.toMap confMap.keys should contain ("spark.master") confMap.keys should contain ("spark.app.name") - confMap.keys should contain ("spark.jars") + confMap.keys should contain (JARS.key) confMap.keys should contain ("spark.driver.memory") confMap.keys should contain ("spark.driver.cores") confMap.keys should contain ("spark.driver.supervise") - confMap.keys should contain ("spark.ui.enabled") - confMap.keys should contain ("spark.submit.deployMode") - conf.get("spark.ui.enabled") should be ("false") + confMap.keys should contain (UI_ENABLED.key) + confMap.keys should contain (SUBMIT_DEPLOY_MODE.key) + conf.get(UI_ENABLED) should be (false) } test("handles standalone client mode") { @@ -401,7 +402,7 @@ class SparkSubmitSuite classpath(0) should endWith ("thejar.jar") conf.get("spark.executor.memory") should be ("5g") conf.get("spark.cores.max") should be ("5") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) } test("handles mesos client mode") { @@ -423,7 +424,7 @@ class SparkSubmitSuite classpath(0) should endWith ("thejar.jar") conf.get("spark.executor.memory") should be ("5g") conf.get("spark.cores.max") should be ("5") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) } test("handles k8s cluster mode") { @@ -466,7 +467,7 @@ class SparkSubmitSuite val (_, _, conf, mainClass) = submit.prepareSubmitEnvironment(appArgs) conf.get("spark.executor.memory") should be ("5g") conf.get("spark.master") should be ("yarn") - conf.get("spark.submit.deployMode") should be ("cluster") + conf.get(SUBMIT_DEPLOY_MODE) should be ("cluster") mainClass should be (SparkSubmit.YARN_CLUSTER_SUBMIT_CLASS) } @@ -661,7 +662,7 @@ class SparkSubmitSuite val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) appArgs.jars should be(Utils.resolveURIs(jars)) appArgs.files should be(Utils.resolveURIs(files)) - conf.get("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) + conf.get(JARS) should be(Utils.resolveURIs(jars + ",thejar.jar").split(",").toSeq) conf.get("spark.files") should be(Utils.resolveURIs(files)) // Test files and archives (Yarn) @@ -691,8 +692,8 @@ class SparkSubmitSuite val appArgs3 = new SparkSubmitArguments(clArgs3) val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) appArgs3.pyFiles should be(Utils.resolveURIs(pyFiles)) - conf3.get("spark.submit.pyFiles") should be( - PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + conf3.get(SUBMIT_PYTHON_FILES) should be( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles))) conf3.get(PYSPARK_DRIVER_PYTHON.key) should be("python3.4") conf3.get(PYSPARK_PYTHON.key) should be("python3.5") } @@ -743,8 +744,8 @@ class SparkSubmitSuite ) val appArgs = new SparkSubmitArguments(clArgs) val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - conf.get("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) - conf.get("spark.files") should be(Utils.resolveURIs(files)) + conf.get(JARS) should be(Utils.resolveURIs(jars + ",thejar.jar").split(",").toSeq) + conf.get(FILES) should be(Utils.resolveURIs(files).split(",").toSeq) // Test files and archives (Yarn) val f2 = File.createTempFile("test-submit-files-archives", "", tmpDir) @@ -775,8 +776,8 @@ class SparkSubmitSuite ) val appArgs3 = new SparkSubmitArguments(clArgs3) val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) - conf3.get("spark.submit.pyFiles") should be( - PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + conf3.get(SUBMIT_PYTHON_FILES) should be( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles))) // Test remote python files val hadoopConf = new Configuration() @@ -797,7 +798,7 @@ class SparkSubmitSuite val appArgs4 = new SparkSubmitArguments(clArgs4) val (_, _, conf4, _) = submit.prepareSubmitEnvironment(appArgs4, conf = Some(hadoopConf)) // Should not format python path for yarn cluster mode - conf4.get("spark.submit.pyFiles") should be(Utils.resolveURIs(remotePyFiles)) + conf4.get(SUBMIT_PYTHON_FILES) should be(Utils.resolveURIs(remotePyFiles).split(",")) } } @@ -1023,7 +1024,7 @@ class SparkSubmitSuite conf.get("spark.repl.local.jars") should (startWith("file:")) // local py files should not be a URI format. - conf.get("spark.submit.pyFiles") should (startWith("/")) + conf.get(SUBMIT_PYTHON_FILES).foreach { _ should (startWith("/")) } } } @@ -1154,7 +1155,7 @@ class SparkSubmitSuite val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) conf.get(PY_FILES.key) should be(s"s3a://${pyFile.getAbsolutePath}") - conf.get("spark.submit.pyFiles") should (startWith("/")) + conf.get(SUBMIT_PYTHON_FILES).foreach { _ should (startWith("/")) } // Verify "spark.submit.pyFiles" val args1 = Seq( @@ -1170,7 +1171,7 @@ class SparkSubmitSuite val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1, conf = Some(hadoopConf)) conf1.get(PY_FILES.key) should be(s"s3a://${pyFile.getAbsolutePath}") - conf1.get("spark.submit.pyFiles") should (startWith("/")) + conf.get(SUBMIT_PYTHON_FILES).foreach { _ should (startWith("/")) } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index a1707e6540b3..baeefea3158e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.client +import java.io.Closeable import java.util.concurrent.ConcurrentLinkedQueue import scala.concurrent.duration._ @@ -85,57 +86,59 @@ class AppClientSuite } test("interface methods of AppClient using local Master") { - val ci = new AppClientInst(masterRpcEnv.address.toSparkURL) + Utils.tryWithResource(new AppClientInst(masterRpcEnv.address.toSparkURL)) { ci => - ci.client.start() + ci.client.start() - // Client should connect with one Master which registers the application - eventually(timeout(10.seconds), interval(10.millis)) { - val apps = getApplications() - assert(ci.listener.connectedIdList.size === 1, "client listener should have one connection") - assert(apps.size === 1, "master should have 1 registered app") - } + // Client should connect with one Master which registers the application + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(ci.listener.connectedIdList.size === 1, "client listener should have one connection") + assert(apps.size === 1, "master should have 1 registered app") + } - // Send message to Master to request Executors, verify request by change in executor limit - val numExecutorsRequested = 1 - whenReady( + // Send message to Master to request Executors, verify request by change in executor limit + val numExecutorsRequested = 1 + whenReady( ci.client.requestTotalExecutors(numExecutorsRequested), timeout(10.seconds), interval(10.millis)) { acknowledged => - assert(acknowledged) - } + assert(acknowledged) + } - eventually(timeout(10.seconds), interval(10.millis)) { - val apps = getApplications() - assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed") - } + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed") + } - // Send request to kill executor, verify request was made - val executorId: String = getApplications().head.executors.head._2.fullId - whenReady( + // Send request to kill executor, verify request was made + val executorId: String = getApplications().head.executors.head._2.fullId + whenReady( ci.client.killExecutors(Seq(executorId)), timeout(10.seconds), interval(10.millis)) { acknowledged => - assert(acknowledged) - } + assert(acknowledged) + } - // Issue stop command for Client to disconnect from Master - ci.client.stop() + // Issue stop command for Client to disconnect from Master + ci.client.stop() - // Verify Client is marked dead and unregistered from Master - eventually(timeout(10.seconds), interval(10.millis)) { - val apps = getApplications() - assert(ci.listener.deadReasonList.size === 1, "client should have been marked dead") - assert(apps.isEmpty, "master should have 0 registered apps") + // Verify Client is marked dead and unregistered from Master + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(ci.listener.deadReasonList.size === 1, "client should have been marked dead") + assert(apps.isEmpty, "master should have 0 registered apps") + } } } test("request from AppClient before initialized with master") { - val ci = new AppClientInst(masterRpcEnv.address.toSparkURL) + Utils.tryWithResource(new AppClientInst(masterRpcEnv.address.toSparkURL)) { ci => - // requests to master should fail immediately - whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success => - assert(success === false) + // requests to master should fail immediately + whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success => + assert(success === false) + } } } @@ -219,13 +222,17 @@ class AppClientSuite } /** Create AppClient and supporting objects */ - private class AppClientInst(masterUrl: String) { + private class AppClientInst(masterUrl: String) extends Closeable { val rpcEnv = RpcEnv.create("spark", Utils.localHostName(), 0, conf, securityManager) private val cmd = new Command(TestExecutor.getClass.getCanonicalName.stripSuffix("$"), List(), Map(), Seq(), Seq(), Seq()) private val desc = new ApplicationDescription("AppClientSuite", Some(1), 512, cmd, "ignored") val listener = new AppClientCollector val client = new StandaloneAppClient(rpcEnv, Array(masterUrl), desc, listener, new SparkConf) + + override def close(): Unit = { + rpcEnv.shutdown() + } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 7d6efd95fbab..74574e2024c1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -44,6 +44,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.DRIVER_LOG_DFS_DIR import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.UI.{ADMIN_ACLS, ADMIN_ACLS_GROUPS, USER_GROUPS_MAPPING} import org.apache.spark.io._ import org.apache.spark.scheduler._ import org.apache.spark.security.GroupMappingServiceProvider @@ -623,6 +624,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc "test", Some("attempt1")), SparkListenerEnvironmentUpdate(Map( "Spark Properties" -> properties.toSeq, + "Hadoop Properties" -> Seq.empty, "JVM Information" -> Seq.empty, "System Properties" -> Seq.empty, "Classpath Entries" -> Seq.empty @@ -644,12 +646,12 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test both history ui admin acls and application acls are configured. val conf1 = createTestConf() - .set(UI_ACLS_ENABLE, true) - .set(UI_ADMIN_ACLS, "user1,user2") - .set(UI_ADMIN_ACLS_GROUPS, "group1") - .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) + .set(HISTORY_SERVER_UI_ADMIN_ACLS, Seq("user1", "user2")) + .set(HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS, Seq("group1")) + .set(USER_GROUPS_MAPPING, classOf[TestGroupsMappingProvider].getName) - createAndCheck(conf1, ("spark.admin.acls", "user"), ("spark.admin.acls.groups", "group")) { + createAndCheck(conf1, (ADMIN_ACLS.key, "user"), (ADMIN_ACLS_GROUPS.key, "group")) { securityManager => // Test whether user has permission to access UI. securityManager.checkUIViewPermissions("user1") should be (true) @@ -666,10 +668,10 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test only history ui admin acls are configured. val conf2 = createTestConf() - .set(UI_ACLS_ENABLE, true) - .set(UI_ADMIN_ACLS, "user1,user2") - .set(UI_ADMIN_ACLS_GROUPS, "group1") - .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) + .set(HISTORY_SERVER_UI_ADMIN_ACLS, Seq("user1", "user2")) + .set(HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS, Seq("group1")) + .set(USER_GROUPS_MAPPING, classOf[TestGroupsMappingProvider].getName) createAndCheck(conf2) { securityManager => // Test whether user has permission to access UI. securityManager.checkUIViewPermissions("user1") should be (true) @@ -686,8 +688,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test neither history ui admin acls nor application acls are configured. val conf3 = createTestConf() - .set(UI_ACLS_ENABLE, true) - .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) + .set(USER_GROUPS_MAPPING, classOf[TestGroupsMappingProvider].getName) createAndCheck(conf3) { securityManager => // Test whether user has permission to access UI. securityManager.checkUIViewPermissions("user1") should be (false) @@ -881,6 +883,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc SparkListenerApplicationStart("end-event-test", Some("end-event-test"), 1L, "test", None), SparkListenerEnvironmentUpdate(Map( "Spark Properties" -> Seq.empty, + "Hadoop Properties" -> Seq.empty, "JVM Information" -> Seq.empty, "System Properties" -> Seq.empty, "Classpath Entries" -> Seq.empty 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 bb7d3c52bc9c..1a071fa77133 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 @@ -48,6 +48,7 @@ import org.apache.spark._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.History._ import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ import org.apache.spark.status.api.v1.ApplicationInfo import org.apache.spark.status.api.v1.JobData import org.apache.spark.ui.SparkUI @@ -613,9 +614,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers stop() init( - "spark.ui.filters" -> classOf[FakeAuthFilter].getName(), - UI_ACLS_ENABLE.key -> "true", - UI_ADMIN_ACLS.key -> admin) + UI_FILTERS.key -> classOf[FakeAuthFilter].getName(), + HISTORY_SERVER_UI_ACLS_ENABLE.key -> "true", + HISTORY_SERVER_UI_ADMIN_ACLS.key -> admin) val tests = Seq( (owner, HttpServletResponse.SC_OK), diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 84b3a29b58bf..fbf2acc3175d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -39,6 +39,10 @@ import other.supplier.{CustomPersistenceEngine, CustomRecoveryModeFactory} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy._ import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Deploy._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.serializer @@ -101,10 +105,9 @@ class MasterSuite extends SparkFunSuite test("can use a custom recovery mode factory") { val conf = new SparkConf(loadDefaults = false) - conf.set("spark.deploy.recoveryMode", "CUSTOM") - conf.set("spark.deploy.recoveryMode.factory", - classOf[CustomRecoveryModeFactory].getCanonicalName) - conf.set("spark.master.rest.enabled", "false") + conf.set(RECOVERY_MODE, "CUSTOM") + conf.set(RECOVERY_MODE_FACTORY, classOf[CustomRecoveryModeFactory].getCanonicalName) + conf.set(MASTER_REST_SERVER_ENABLED, false) val instantiationAttempts = CustomRecoveryModeFactory.instantiationAttempts @@ -186,10 +189,9 @@ class MasterSuite extends SparkFunSuite test("master correctly recover the application") { val conf = new SparkConf(loadDefaults = false) - conf.set("spark.deploy.recoveryMode", "CUSTOM") - conf.set("spark.deploy.recoveryMode.factory", - classOf[FakeRecoveryModeFactory].getCanonicalName) - conf.set("spark.master.rest.enabled", "false") + conf.set(RECOVERY_MODE, "CUSTOM") + conf.set(RECOVERY_MODE_FACTORY, classOf[FakeRecoveryModeFactory].getCanonicalName) + conf.set(MASTER_REST_SERVER_ENABLED, false) val fakeAppInfo = makeAppInfo(1024) val fakeWorkerInfo = makeWorkerInfo(8192, 16) @@ -286,8 +288,8 @@ class MasterSuite extends SparkFunSuite implicit val formats = org.json4s.DefaultFormats val reverseProxyUrl = "http://localhost:8080" val conf = new SparkConf() - conf.set("spark.ui.reverseProxy", "true") - conf.set("spark.ui.reverseProxyUrl", reverseProxyUrl) + conf.set(UI_REVERSE_PROXY, true) + conf.set(UI_REVERSE_PROXY_URL, reverseProxyUrl) val localCluster = new LocalSparkCluster(2, 2, 512, conf) localCluster.start() try { @@ -635,66 +637,77 @@ class MasterSuite extends SparkFunSuite } test("SPARK-19900: there should be a corresponding driver for the app after relaunching driver") { - val conf = new SparkConf().set("spark.worker.timeout", "1") + val conf = new SparkConf().set(WORKER_TIMEOUT, 1L) val master = makeMaster(conf) master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) eventually(timeout(10.seconds)) { val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") } - val worker1 = new MockWorker(master.self) - worker1.rpcEnv.setupEndpoint("worker", worker1) - val worker1Reg = RegisterWorker( - worker1.id, - "localhost", - 9998, - worker1.self, - 10, - 1024, - "http://localhost:8080", - RpcAddress("localhost2", 10000)) - master.self.send(worker1Reg) - val driver = DeployTestUtils.createDriverDesc().copy(supervise = true) - master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver)) - - eventually(timeout(10.seconds)) { - assert(worker1.apps.nonEmpty) - } - - eventually(timeout(10.seconds)) { - val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) - assert(masterState.workers(0).state == WorkerState.DEAD) - } + var worker1: MockWorker = null + var worker2: MockWorker = null + try { + worker1 = new MockWorker(master.self) + worker1.rpcEnv.setupEndpoint("worker", worker1) + val worker1Reg = RegisterWorker( + worker1.id, + "localhost", + 9998, + worker1.self, + 10, + 1024, + "http://localhost:8080", + RpcAddress("localhost2", 10000)) + master.self.send(worker1Reg) + val driver = DeployTestUtils.createDriverDesc().copy(supervise = true) + master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver)) + + eventually(timeout(10.seconds)) { + assert(worker1.apps.nonEmpty) + } - val worker2 = new MockWorker(master.self) - worker2.rpcEnv.setupEndpoint("worker", worker2) - master.self.send(RegisterWorker( - worker2.id, - "localhost", - 9999, - worker2.self, - 10, - 1024, - "http://localhost:8081", - RpcAddress("localhost", 10001))) - eventually(timeout(10.seconds)) { - assert(worker2.apps.nonEmpty) - } + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.workers(0).state == WorkerState.DEAD) + } - master.self.send(worker1Reg) - eventually(timeout(10.seconds)) { - val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + worker2 = new MockWorker(master.self) + worker2.rpcEnv.setupEndpoint("worker", worker2) + master.self.send(RegisterWorker( + worker2.id, + "localhost", + 9999, + worker2.self, + 10, + 1024, + "http://localhost:8081", + RpcAddress("localhost", 10001))) + eventually(timeout(10.seconds)) { + assert(worker2.apps.nonEmpty) + } - val worker = masterState.workers.filter(w => w.id == worker1.id) - assert(worker.length == 1) - // make sure the `DriverStateChanged` arrives at Master. - assert(worker(0).drivers.isEmpty) - assert(worker1.apps.isEmpty) - assert(worker1.drivers.isEmpty) - assert(worker2.apps.size == 1) - assert(worker2.drivers.size == 1) - assert(masterState.activeDrivers.length == 1) - assert(masterState.activeApps.length == 1) + master.self.send(worker1Reg) + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + + val worker = masterState.workers.filter(w => w.id == worker1.id) + assert(worker.length == 1) + // make sure the `DriverStateChanged` arrives at Master. + assert(worker(0).drivers.isEmpty) + assert(worker1.apps.isEmpty) + assert(worker1.drivers.isEmpty) + assert(worker2.apps.size == 1) + assert(worker2.drivers.size == 1) + assert(masterState.activeDrivers.length == 1) + assert(masterState.activeApps.length == 1) + } + } finally { + if (worker1 != null) { + worker1.rpcEnv.shutdown() + } + if (worker2 != null) { + worker2.rpcEnv.shutdown() + } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala index 30278655dbe0..3d8a46bd02e1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala @@ -24,6 +24,7 @@ import org.apache.commons.lang3.RandomUtils import org.apache.curator.test.TestingServer import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Deploy.ZOOKEEPER_URL import org.apache.spark.rpc.{RpcEndpoint, RpcEnv} import org.apache.spark.serializer.{JavaSerializer, Serializer} import org.apache.spark.util.Utils @@ -48,7 +49,7 @@ class PersistenceEngineSuite extends SparkFunSuite { val zkTestServer = new TestingServer(findFreePort(conf)) try { testPersistenceEngine(conf, serializer => { - conf.set("spark.deploy.zookeeper.url", zkTestServer.getConnectString) + conf.set(ZOOKEEPER_URL, zkTestServer.getConnectString) new ZooKeeperPersistenceEngine(conf, serializer) }) } finally { diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala index 75c50af23c66..87655f310f1e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -22,6 +22,7 @@ import java.lang.Boolean import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ import org.apache.spark.util.Utils /** @@ -93,7 +94,7 @@ class SubmitRestProtocolSuite extends SparkFunSuite { message.sparkProperties = conf.getAll.toMap message.validate() // optional fields - conf.set("spark.jars", "mayonnaise.jar,ketchup.jar") + conf.set(JARS, Seq("mayonnaise.jar", "ketchup.jar")) conf.set("spark.files", "fireball.png") conf.set("spark.driver.memory", s"${Utils.DEFAULT_DRIVER_MEM_MB}m") conf.set("spark.driver.cores", "180") diff --git a/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala index 18aa537b3a51..daa7e544cc9c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/KafkaTokenUtilSuite.scala @@ -17,9 +17,11 @@ package org.apache.spark.deploy.security -import java.{ util => ju } +import java.{util => ju} +import java.security.PrivilegedExceptionAction import javax.security.auth.login.{AppConfigurationEntry, Configuration} +import org.apache.hadoop.security.UserGroupInformation import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} @@ -78,6 +80,21 @@ class KafkaTokenUtilSuite extends SparkFunSuite with BeforeAndAfterEach { Configuration.setConfiguration(null) } + test("checkProxyUser with proxy current user should throw exception") { + val realUser = UserGroupInformation.createUserForTesting("realUser", Array()) + UserGroupInformation.createProxyUserForTesting("proxyUser", realUser, Array()).doAs( + new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val thrown = intercept[IllegalArgumentException] { + KafkaTokenUtil.checkProxyUser() + } + assert(thrown.getMessage contains + "Obtaining delegation token for proxy user is not yet supported.") + } + } + ) + } + test("createAdminClientProperties without bootstrap servers should throw exception") { val thrown = intercept[IllegalArgumentException] { KafkaTokenUtil.createAdminClientProperties(sparkConf) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index e5e5b5e428c4..0ddf38c04409 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.{Command, ExecutorState, ExternalShuffleService} import org.apache.spark.deploy.DeployMessages.{DriverStateChanged, ExecutorStateChanged} import org.apache.spark.deploy.master.DriverState +import org.apache.spark.internal.config.Worker._ import org.apache.spark.rpc.{RpcAddress, RpcEnv} class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { @@ -100,7 +101,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedExecutors (small number of executors)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedExecutors", 2.toString) + conf.set(WORKER_UI_RETAINED_EXECUTORS, 2) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 5) { @@ -124,7 +125,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedExecutors (more executors)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedExecutors", 30.toString) + conf.set(WORKER_UI_RETAINED_EXECUTORS, 30) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 50) { @@ -157,7 +158,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedDrivers (small number of drivers)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedDrivers", 2.toString) + conf.set(WORKER_UI_RETAINED_DRIVERS, 2) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 5) { @@ -181,7 +182,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedDrivers (more drivers)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedDrivers", 30.toString) + conf.set(WORKER_UI_RETAINED_DRIVERS, 30) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 50) { diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index a5fe2026c0f7..558cd3626ab9 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -41,6 +41,7 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.memory.TestMemoryManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rdd.RDD @@ -169,7 +170,7 @@ class ExecutorSuite extends SparkFunSuite val conf = new SparkConf() .setMaster("local") .setAppName("executor thread test") - .set("spark.ui.enabled", "false") + .set(UI_ENABLED.key, "false") sc = new SparkContext(conf) val executorThread = sc.parallelize(Seq(1), 1).map { _ => Thread.currentThread.getClass.getName diff --git a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala index c88cc13654ce..548949edf47b 100644 --- a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.util.Utils class LauncherBackendSuite extends SparkFunSuite with Matchers { @@ -48,7 +49,7 @@ class LauncherBackendSuite extends SparkFunSuite with Matchers { val handle = new SparkLauncher(env) .setSparkHome(sys.props("spark.test.home")) .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) - .setConf("spark.ui.enabled", "false") + .setConf(UI_ENABLED.key, "false") .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, s"-Dtest.appender=console") .setMaster(master) .setAppResource(SparkLauncher.NO_RESOURCE) diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 8556e920daeb..0a689f81a576 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -43,10 +43,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes maxOnHeapExecutionMemory: Long, maxOffHeapExecutionMemory: Long): UnifiedMemoryManager = { val conf = new SparkConf() - .set("spark.memory.fraction", "1") + .set(MEMORY_FRACTION, 1.0) .set(TEST_MEMORY, maxOnHeapExecutionMemory) .set(MEMORY_OFFHEAP_SIZE, maxOffHeapExecutionMemory) - .set("spark.memory.storageFraction", storageFraction.toString) + .set(MEMORY_STORAGE_FRACTION, storageFraction) UnifiedMemoryManager(conf, numCores = 1) } @@ -223,9 +223,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes val reservedMemory = 300L * 1024 val memoryFraction = 0.8 val conf = new SparkConf() - .set("spark.memory.fraction", memoryFraction.toString) + .set(MEMORY_FRACTION, memoryFraction) .set(TEST_MEMORY, systemMemory) .set(TEST_RESERVED_MEMORY, reservedMemory) + val mm = UnifiedMemoryManager(conf, numCores = 1) val expectedMaxMemory = ((systemMemory - reservedMemory) * memoryFraction).toLong assert(mm.maxHeapMemory === expectedMaxMemory) @@ -243,9 +244,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes val reservedMemory = 300L * 1024 val memoryFraction = 0.8 val conf = new SparkConf() - .set("spark.memory.fraction", memoryFraction.toString) + .set(MEMORY_FRACTION, memoryFraction) .set(TEST_MEMORY, systemMemory) .set(TEST_RESERVED_MEMORY, reservedMemory) + val mm = UnifiedMemoryManager(conf, numCores = 1) // Try using an executor memory that's too small @@ -258,9 +260,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes test("execution can evict cached blocks when there are multiple active tasks (SPARK-12155)") { val conf = new SparkConf() - .set("spark.memory.fraction", "1") - .set("spark.memory.storageFraction", "0") + .set(MEMORY_FRACTION, 1.0) + .set(MEMORY_STORAGE_FRACTION, 0.0) .set(TEST_MEMORY, 1000L) + val mm = UnifiedMemoryManager(conf, numCores = 2) val ms = makeMemoryStore(mm) val memoryMode = MemoryMode.ON_HEAP @@ -284,9 +287,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes test("SPARK-15260: atomically resize memory pools") { val conf = new SparkConf() - .set("spark.memory.fraction", "1") - .set("spark.memory.storageFraction", "0") + .set(MEMORY_FRACTION, 1.0) + .set(MEMORY_STORAGE_FRACTION, 0.0) .set(TEST_MEMORY, 1000L) + val mm = UnifiedMemoryManager(conf, numCores = 2) makeBadMemoryStore(mm) val memoryMode = MemoryMode.ON_HEAP diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala index a85011b42bbc..800fc1e4a3f1 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.METRICS_CONF class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { var filePath: String = _ @@ -31,7 +32,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { test("MetricsConfig with default properties") { val sparkConf = new SparkConf(loadDefaults = false) - sparkConf.set("spark.metrics.conf", "dummy-file") + sparkConf.set(METRICS_CONF, "dummy-file") val conf = new MetricsConfig(sparkConf) conf.initialize() @@ -47,7 +48,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { test("MetricsConfig with properties set from a file") { val sparkConf = new SparkConf(loadDefaults = false) - sparkConf.set("spark.metrics.conf", filePath) + sparkConf.set(METRICS_CONF, filePath) val conf = new MetricsConfig(sparkConf) conf.initialize() @@ -110,7 +111,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { setMetricsProperty(sparkConf, "*.source.jvm.class", "org.apache.spark.SomeOtherSource") setMetricsProperty(sparkConf, "master.sink.console.period", "50") setMetricsProperty(sparkConf, "master.sink.console.unit", "seconds") - sparkConf.set("spark.metrics.conf", filePath) + sparkConf.set(METRICS_CONF, filePath) val conf = new MetricsConfig(sparkConf) conf.initialize() @@ -135,7 +136,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { test("MetricsConfig with subProperties") { val sparkConf = new SparkConf(loadDefaults = false) - sparkConf.set("spark.metrics.conf", filePath) + sparkConf.set(METRICS_CONF, filePath) val conf = new MetricsConfig(sparkConf) conf.initialize() diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index a7a24114f17e..c512f29c8442 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -35,7 +35,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile - conf = new SparkConf(false).set("spark.metrics.conf", filePath) + conf = new SparkConf(false).set(METRICS_CONF, filePath) securityMgr = new SecurityManager(conf) } 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 21138bd4a16b..f1cf14de1f87 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 @@ -32,6 +32,7 @@ import org.scalatest.Matchers import org.scalatest.mockito.MockitoSugar import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ import org.apache.spark.network.{BlockDataManager, BlockTransferService} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.BlockFetchingListener @@ -50,8 +51,8 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security on same password") { val conf = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") testConnection(conf, conf) match { case Success(_) => // expected @@ -61,10 +62,10 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security on mismatch password") { val conf0 = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.authenticate.secret", "bad") + val conf1 = conf0.clone.set(AUTH_SECRET, "bad") testConnection(conf0, conf1) match { case Success(_) => fail("Should have failed") case Failure(t) => t.getMessage should include ("Mismatched response") @@ -73,10 +74,10 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security mismatch auth off on server") { val conf0 = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.authenticate", "false") + val conf1 = conf0.clone.set(NETWORK_AUTH_ENABLED, false) testConnection(conf0, conf1) match { case Success(_) => fail("Should have failed") case Failure(t) => // any funny error may occur, sever will interpret SASL token as RPC @@ -85,10 +86,10 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security mismatch auth off on client") { val conf0 = new SparkConf() - .set("spark.authenticate", "false") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, false) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.authenticate", "true") + val conf1 = conf0.clone.set(NETWORK_AUTH_ENABLED, true) testConnection(conf0, conf1) match { case Success(_) => fail("Should have failed") case Failure(t) => t.getMessage should include ("Expected SaslMessage") @@ -97,8 +98,8 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security with aes encryption") { val conf = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") .set("spark.network.crypto.enabled", "true") .set("spark.network.crypto.saslFallback", "false") diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 2227698cf1ad..e957340826e8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -29,6 +29,7 @@ import org.apache.hadoop.mapred.{FileSplit, TextInputFormat} import org.apache.spark._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.rdd.RDDSuiteUtils._ import org.apache.spark.util.{ThreadUtils, Utils} @@ -136,10 +137,10 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(serialUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === false) - sc.conf.set("spark.rdd.parallelListingThreshold", "1") + sc.conf.set(RDD_PARALLEL_LISTING_THRESHOLD, 1) val parallelUnion = sc.union(nums1, nums2) val actual = parallelUnion.collect().toList - sc.conf.remove("spark.rdd.parallelListingThreshold") + sc.conf.remove(RDD_PARALLEL_LISTING_THRESHOLD.key) assert(parallelUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === true) assert(expected === actual) 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 558b7fa49832..51bf5c273f04 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -36,6 +36,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.{SecurityManager, SparkConf, SparkEnv, SparkException, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.config._ import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -693,42 +694,42 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { test("send with authentication") { testSend(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good")) } test("send with SASL encryption") { testSend(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") - .set("spark.authenticate.enableSaslEncryption", "true")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") + .set(SASL_ENCRYPTION_ENABLED, true)) } test("send with AES encryption") { testSend(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.network.crypto.enabled", "true") .set("spark.network.crypto.saslFallback", "false")) } test("ask with authentication") { testAsk(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good")) } test("ask with SASL encryption") { testAsk(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") - .set("spark.authenticate.enableSaslEncryption", "true")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") + .set(SASL_ENCRYPTION_ENABLED, true)) } test("ask with AES encryption") { testAsk(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.network.crypto.enabled", "true") .set("spark.network.crypto.saslFallback", "false")) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 2215f7f36621..aadddca3a560 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -85,7 +85,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM extraConfs = Seq( config.BLACKLIST_ENABLED.key -> "true", // just to avoid this test taking too long - "spark.locality.wait" -> "10ms" + config.LOCALITY_WAIT.key -> "10ms" ) ) { val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 04987e6ef79e..811b9fe46fdf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -108,8 +108,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val secretPassword = "secret_password" val conf = getLoggingConf(testDirPath, None) .set(key, secretPassword) + val hadoopconf = SparkHadoopUtil.get.newConfiguration(new SparkConf()) val eventLogger = new EventLoggingListener("test", None, testDirPath.toUri(), conf) - val envDetails = SparkEnv.environmentDetails(conf, "FIFO", Seq.empty, Seq.empty) + val envDetails = SparkEnv.environmentDetails(conf, hadoopconf, "FIFO", Seq.empty, Seq.empty) val event = SparkListenerEnvironmentUpdate(envDetails) val redactedProps = eventLogger.redactEvent(event).environmentDetails("Spark Properties").toMap assert(redactedProps(key) == "*********(redacted)") @@ -533,7 +534,7 @@ object EventLoggingListenerSuite { conf.set(EVENT_LOG_DIR, logDir.toString) compressionCodec.foreach { codec => conf.set(EVENT_LOG_COMPRESS, true) - conf.set("spark.io.compression.codec", codec) + conf.set(IO_COMPRESSION_CODEC, codec) } conf.set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) conf diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index f41ffb7f2c0b..c1e7fb9a1db1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -181,7 +181,7 @@ class MapStatusSuite extends SparkFunSuite { test("SPARK-21133 HighlyCompressedMapStatus#writeExternal throws NPE") { val conf = new SparkConf() - .set("spark.serializer", classOf[KryoSerializer].getName) + .set(config.SERIALIZER, classOf[KryoSerializer].getName) .setMaster("local") .setAppName("SPARK-21133") withSpark(new SparkContext(conf)) { sc => 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 3bfc97b80184..27369759fad5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.executor.{Executor, TaskMetrics, TaskMetricsSuite} +import org.apache.spark.internal.config.METRICS_CONF import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.JvmSource import org.apache.spark.network.util.JavaUtils @@ -37,7 +38,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark test("provide metrics sources") { val filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile val conf = new SparkConf(loadDefaults = false) - .set("spark.metrics.conf", filePath) + .set(METRICS_CONF, filePath) sc = new SparkContext("local", "test", conf) val rdd = sc.makeRDD(1 to 1) val result = sc.runJob(rdd, (tc: TaskContext, it: Iterator[Int]) => { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 9c555a923d62..85c87b95f7c2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -1011,7 +1011,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("Locality should be used for bulk offers even with delay scheduling off") { val conf = new SparkConf() - .set("spark.locality.wait", "0") + .set(config.LOCALITY_WAIT.key, "0") sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) // we create a manual clock just so we can be sure the clock doesn't advance at all in this test val clock = new ManualClock() @@ -1058,7 +1058,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("With delay scheduling off, tasks can be run at any locality level immediately") { val conf = new SparkConf() - .set("spark.locality.wait", "0") + .set(config.LOCALITY_WAIT.key, "0") sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) // we create a manual clock just so we can be sure the clock doesn't advance at all in this test 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 f9dfd2c456c5..502a013193aa 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -166,7 +166,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg private val conf = new SparkConf - val LOCALITY_WAIT_MS = conf.getTimeAsMs("spark.locality.wait", "3s") + val LOCALITY_WAIT_MS = conf.get(config.LOCALITY_WAIT) val MAX_TASK_FAILURES = 4 var sched: FakeTaskScheduler = null @@ -429,7 +429,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg set(config.BLACKLIST_ENABLED, true). set(config.BLACKLIST_TIMEOUT_CONF, rescheduleDelay). // don't wait to jump locality levels in this test - set("spark.locality.wait", "0") + set(config.LOCALITY_WAIT.key, "0") sc = new SparkContext("local", "test", conf) // two executors on same host, one on different. diff --git a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala index e5d1bf4fde9e..abccb8e9bbf2 100644 --- a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala @@ -75,8 +75,8 @@ class CryptoStreamUtilsSuite extends SparkFunSuite { test("serializer manager integration") { val conf = createConf() - .set("spark.shuffle.compress", "true") - .set("spark.shuffle.spill.compress", "true") + .set(SHUFFLE_COMPRESS, true) + .set(SHUFFLE_SPILL_COMPRESS, true) val plainStr = "hello world" val blockId = new TempShuffleBlockId(UUID.randomUUID()) diff --git a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala index 3734f1cb408f..8610b18702ec 100644 --- a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala @@ -25,9 +25,10 @@ import org.apache.avro.{Schema, SchemaBuilder} import org.apache.avro.generic.GenericData.Record import org.apache.spark.{SharedSparkContext, SparkFunSuite} +import org.apache.spark.internal.config.SERIALIZER class GenericAvroSerializerSuite extends SparkFunSuite with SharedSparkContext { - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") val schema : Schema = SchemaBuilder .record("testRecord").fields() diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala index d7730f23da10..fd228cded783 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala @@ -22,6 +22,8 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoTest._ /** @@ -122,9 +124,9 @@ object KryoBenchmark extends BenchmarkBase { def createSerializer(useUnsafe: Boolean): SerializerInstance = { val conf = new SparkConf() - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) - conf.set("spark.kryo.unsafe", useUnsafe.toString) + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USE_UNSAFE, useUnsafe) new KryoSerializer(conf).newInstance() } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala index 2a15c6f6a2d9..2915b99dcfb6 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala @@ -23,6 +23,8 @@ import scala.concurrent.duration._ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoTest._ import org.apache.spark.util.ThreadUtils @@ -69,9 +71,9 @@ object KryoSerializerBenchmark extends BenchmarkBase { def createSparkContext(usePool: Boolean): SparkContext = { val conf = new SparkConf() - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) - conf.set("spark.kryo.pool", usePool.toString) + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USE_POOL, usePool) if (sc != null) { sc.stop() diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 46aa9c37986c..ae871091ba3c 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -28,8 +28,8 @@ class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContex test("kryo objects are serialised consistently in different processes") { val conf = new SparkConf(false) - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) + .set(config.SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + .set(config.Kryo.KRYO_USER_REGISTRATORS, classOf[AppJarRegistrator].getName) .set(config.MAX_TASK_FAILURES, 1) .set(config.BLACKLIST_ENABLED, false) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala index cf01f79f4909..25f0b19c980f 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala @@ -21,6 +21,8 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.LocalSparkContext._ import org.apache.spark.SparkContext import org.apache.spark.SparkException +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ class KryoSerializerResizableOutputSuite extends SparkFunSuite { @@ -29,9 +31,9 @@ class KryoSerializerResizableOutputSuite extends SparkFunSuite { test("kryo without resizable output buffer should fail on large array") { val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer", "1m") - conf.set("spark.kryoserializer.buffer.max", "1m") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") + conf.set(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "1m") withSpark(new SparkContext("local", "test", conf)) { sc => intercept[SparkException](sc.parallelize(x).collect()) } @@ -39,9 +41,9 @@ class KryoSerializerResizableOutputSuite extends SparkFunSuite { test("kryo with resizable output buffer should succeed on large array") { val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer", "1m") - conf.set("spark.kryoserializer.buffer.max", "2m") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") + conf.set(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "2m") withSpark(new SparkContext("local", "test", conf)) { sc => assert(sc.parallelize(x).collect() === x) } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 8af53274d9b2..16eec7e0bea1 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -32,19 +32,21 @@ import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import org.roaringbitmap.RoaringBitmap import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{ThreadUtils, Utils} class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) - conf.set("spark.kryo.unsafe", "false") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USE_UNSAFE, false) test("SPARK-7392 configuration limits") { - val kryoBufferProperty = "spark.kryoserializer.buffer" - val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max" + val kryoBufferProperty = KRYO_SERIALIZER_BUFFER_SIZE.key + val kryoBufferMaxProperty = KRYO_SERIALIZER_MAX_BUFFER_SIZE.key def newKryoInstance( conf: SparkConf, @@ -81,7 +83,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("basic types") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { @@ -114,7 +116,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("pairs") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { @@ -141,7 +143,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("Scala data structures") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { @@ -169,7 +171,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } test("Bug: SPARK-10251") { - val ser = new KryoSerializer(conf.clone.set("spark.kryo.registrationRequired", "true")) + val ser = new KryoSerializer(conf.clone.set(KRYO_REGISTRATION_REQUIRED, true)) .newInstance() def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) @@ -253,7 +255,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { hashMap.put("foo", "bar") check(hashMap) - System.clearProperty("spark.kryo.registrator") + System.clearProperty(KRYO_USER_REGISTRATORS.key) } test("kryo with collect") { @@ -310,7 +312,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { import org.apache.spark.SparkException val conf = new SparkConf(false) - conf.set("spark.kryo.registrator", "this.class.does.not.exist") + conf.set(KRYO_USER_REGISTRATORS, "this.class.does.not.exist") val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance().serialize(1)) assert(thrown.getMessage.contains("Failed to register classes with Kryo")) @@ -337,7 +339,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("registration of HighlyCompressedMapStatus") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) // these cases require knowing the internals of RoaringBitmap a little. Blocks span 2^16 // values, and they use a bitmap (dense) if they have more than 4096 values, and an @@ -355,7 +357,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("serialization buffer overflow reporting") { import org.apache.spark.SparkException - val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max" + val kryoBufferMaxProperty = KRYO_SERIALIZER_MAX_BUFFER_SIZE.key val largeObject = (1 to 1000000).toArray @@ -409,7 +411,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("getAutoReset") { val ser = new KryoSerializer(new SparkConf).newInstance().asInstanceOf[KryoSerializerInstance] assert(ser.getAutoReset) - val conf = new SparkConf().set("spark.kryo.registrator", + val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) val ser2 = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance] assert(!ser2.getAutoReset) @@ -438,10 +440,10 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { private def testSerializerInstanceReuse( autoReset: Boolean, referenceTracking: Boolean, usePool: Boolean): Unit = { val conf = new SparkConf(loadDefaults = false) - .set("spark.kryo.referenceTracking", referenceTracking.toString) - .set("spark.kryo.pool", usePool.toString) + .set(KRYO_REFERENCE_TRACKING, referenceTracking) + .set(KRYO_USE_POOL, usePool) if (!autoReset) { - conf.set("spark.kryo.registrator", classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) } val ser = new KryoSerializer(conf) val serInstance = ser.newInstance().asInstanceOf[KryoSerializerInstance] @@ -478,7 +480,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { implicit val executionContext: ExecutionContext = ExecutionContext.fromExecutor( Executors.newFixedThreadPool(4)) - val ser = new KryoSerializer(conf.clone.set("spark.kryo.pool", "true")) + val ser = new KryoSerializer(conf.clone.set(KRYO_USE_POOL, true)) val tests = mutable.ListBuffer[Future[Boolean]]() @@ -519,11 +521,11 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSparkContext { - conf.set("spark.serializer", classOf[KryoSerializer].getName) - conf.set("spark.kryo.registrator", classOf[RegistratorWithoutAutoReset].getName) - conf.set("spark.kryo.referenceTracking", "true") - conf.set("spark.shuffle.manager", "sort") - conf.set("spark.shuffle.sort.bypassMergeThreshold", "200") + conf.set(SERIALIZER, classOf[KryoSerializer].getName) + conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_REFERENCE_TRACKING, true) + conf.set(SHUFFLE_MANAGER, "sort") + conf.set(SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD, 200) test("sort-shuffle with bypassMergeSort (SPARK-7873)") { val myObject = ("Hello", "World") diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala index 99882bf76e29..dad080c5fc16 100644 --- a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala @@ -24,6 +24,7 @@ import scala.util.Random import org.scalatest.Assertions import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoTest.RegistratorWithoutAutoReset /** @@ -50,7 +51,7 @@ class SerializerPropertiesSuite extends SparkFunSuite { } test("KryoSerializer does not support relocation when auto-reset is disabled") { - val conf = new SparkConf().set("spark.kryo.registrator", + val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) val ser = new KryoSerializer(conf) assert(!ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset()) diff --git a/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala index d63a45ae4a6a..126ba0e8b1e9 100644 --- a/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala @@ -17,17 +17,19 @@ package org.apache.spark.serializer +import org.apache.spark.internal.config.Kryo._ + class UnsafeKryoSerializerSuite extends KryoSerializerSuite { // This test suite should run all tests in KryoSerializerSuite with kryo unsafe. override def beforeAll() { - conf.set("spark.kryo.unsafe", "true") + conf.set(KRYO_USE_UNSAFE, true) super.beforeAll() } override def afterAll() { - conf.set("spark.kryo.unsafe", "false") + conf.set(KRYO_USE_UNSAFE, false) super.afterAll() } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index eb97d5a1e507..6d2ef17a7a79 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -23,6 +23,7 @@ import java.nio.ByteBuffer import org.mockito.Mockito.{mock, when} import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} @@ -123,8 +124,8 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext val serializerManager = new SerializerManager( serializer, new SparkConf() - .set("spark.shuffle.compress", "false") - .set("spark.shuffle.spill.compress", "false")) + .set(config.SHUFFLE_COMPRESS, false) + .set(config.SHUFFLE_SPILL_COMPRESS, false)) val taskContext = TaskContext.empty() val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala index 43621cb85762..9d71647d082c 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.internal.config.MEMORY_FRACTION import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory._ import org.apache.spark.unsafe.Platform @@ -36,7 +37,8 @@ class ShuffleExternalSorterSuite extends SparkFunSuite with LocalSparkContext wi .setAppName("ShuffleExternalSorterSuite") .set(IS_TESTING, true) .set(TEST_MEMORY, 1600L) - .set("spark.memory.fraction", "1") + .set(MEMORY_FRACTION, 1.0) + sc = new SparkContext(conf) val memoryManager = UnifiedMemoryManager(conf, 1) 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 2250ae2f771e..10855bf59c3a 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{DRIVER_PORT, MEMORY_OFFHEAP_SIZE} +import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.BlockTransferService @@ -85,18 +85,17 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite before { rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) - - conf.set("spark.authenticate", "false") + conf.set(NETWORK_AUTH_ENABLED, false) conf.set(DRIVER_PORT, rpcEnv.address.port) conf.set(IS_TESTING, true) - conf.set("spark.memory.fraction", "1") - conf.set("spark.memory.storageFraction", "1") - conf.set("spark.storage.unrollMemoryThreshold", "512") + conf.set(MEMORY_FRACTION, 1.0) + conf.set(MEMORY_STORAGE_FRACTION, 1.0) + conf.set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) // to make a replication attempt to inactive store fail fast conf.set("spark.core.connection.ack.wait.timeout", "1s") // to make cached peers refresh frequently - conf.set("spark.storage.cachedPeersTtl", "10") + conf.set(STORAGE_CACHED_PEERS_TTL, 10) sc = new SparkContext("local", "test", conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", @@ -424,8 +423,8 @@ class BlockManagerReplicationSuite extends BlockManagerReplicationBehavior { class BlockManagerProactiveReplicationSuite extends BlockManagerReplicationBehavior { val conf = new SparkConf(false).set("spark.app.id", "test") conf.set("spark.kryoserializer.buffer", "1m") - conf.set("spark.storage.replication.proactive", "true") - conf.set("spark.storage.exceptionOnPinLeak", "true") + conf.set(STORAGE_REPLICATION_PROACTIVE, true) + conf.set(STORAGE_EXCEPTION_PIN_LEAK, true) (2 to 5).foreach { i => test(s"proactive block replication - $i replicas - ${i - 1} block manager deletions") { @@ -500,10 +499,10 @@ class BlockManagerBasicStrategyReplicationSuite extends BlockManagerReplicationB val conf: SparkConf = new SparkConf(false).set("spark.app.id", "test") conf.set("spark.kryoserializer.buffer", "1m") conf.set( - "spark.storage.replication.policy", + STORAGE_REPLICATION_POLICY, classOf[BasicBlockReplicationPolicy].getName) conf.set( - "spark.storage.replication.topologyMapper", + STORAGE_REPLICATION_TOPOLOGY_MAPPER, classOf[DummyTopologyMapper].getName) } 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 c23264191e12..04de0e41a341 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -46,7 +46,6 @@ import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransport import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} -import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} @@ -66,9 +65,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE implicit val defaultSignaler: Signaler = ThreadSignaler var conf: SparkConf = null - var store: BlockManager = null - var store2: BlockManager = null - var store3: BlockManager = null + val allStores = ArrayBuffer[BlockManager]() var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null val securityMgr = new SecurityManager(new SparkConf(false)) @@ -106,6 +103,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, bmConf, memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, 0) memManager.setMemoryStore(blockManager.memoryStore) + allStores += blockManager blockManager.initialize("app-id") blockManager } @@ -117,10 +115,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf = new SparkConf(false) .set("spark.app.id", "test") .set(IS_TESTING, true) - .set("spark.memory.fraction", "1") - .set("spark.memory.storageFraction", "1") + .set(MEMORY_FRACTION, 1.0) + .set(MEMORY_STORAGE_FRACTION, 1.0) .set("spark.kryoserializer.buffer", "1m") - .set("spark.storage.unrollMemoryThreshold", "512") + .set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set(DRIVER_PORT, rpcEnv.address.port) @@ -140,18 +138,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE override def afterEach(): Unit = { try { conf = null - if (store != null) { - store.stop() - store = null - } - if (store2 != null) { - store2.stop() - store2 = null - } - if (store3 != null) { - store3.stop() - store3 = null - } + allStores.foreach(_.stop()) + allStores.clear() rpcEnv.shutdown() rpcEnv.awaitTermination() rpcEnv = null @@ -161,6 +149,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } + private def stopBlockManager(blockManager: BlockManager): Unit = { + allStores -= blockManager + blockManager.stop() + } + test("StorageLevel object caching") { val level1 = StorageLevel(false, false, false, 3) // this should return the same object as level1 @@ -204,7 +197,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("master + 1 manager interaction") { - store = makeBlockManager(20000) + val store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -234,8 +227,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("master + 2 managers interaction") { - store = makeBlockManager(2000, "exec1") - store2 = makeBlockManager(2000, "exec2") + val store = makeBlockManager(2000, "exec1") + val store2 = makeBlockManager(2000, "exec2") val peers = master.getPeers(store.blockManagerId) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -250,7 +243,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("removing block") { - store = makeBlockManager(20000) + val store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -298,7 +291,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("removing rdd") { - store = makeBlockManager(20000) + val store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -331,7 +324,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("removing broadcast") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val driverStore = store val executorStore = makeBlockManager(2000, "executor") val a1 = new Array[Byte](400) @@ -397,11 +390,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } executorStore.stop() driverStore.stop() - store = null } test("reregistration on heart beat") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -418,7 +410,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("reregistration on block update") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -436,7 +428,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("reregistration doesn't dead lock") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -474,7 +466,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("correct BlockResult returned from get() calls") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) @@ -545,27 +537,25 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("SPARK-9591: getRemoteBytes from another location when Exception throw") { conf.set("spark.shuffle.io.maxRetries", "0") - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") - store3 = makeBlockManager(8000, "executor3") + val store = makeBlockManager(8000, "executor1") + val store2 = makeBlockManager(8000, "executor2") + val store3 = makeBlockManager(8000, "executor3") val list1 = List(new Array[Byte](4000)) store2.putIterator( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store3.putIterator( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") - store2.stop() - store2 = null + stopBlockManager(store2) assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") - store3.stop() - store3 = null + stopBlockManager(store3) // Should return None instead of throwing an exception: assert(store.getRemoteBytes("list1").isEmpty) } test("SPARK-14252: getOrElseUpdate should still read from remote storage") { - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") + val store = makeBlockManager(8000, "executor1") + val store2 = makeBlockManager(8000, "executor2") val list1 = List(new Array[Byte](4000)) store2.putIterator( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) @@ -593,7 +583,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } private def testInMemoryLRUStorage(storageLevel: StorageLevel): Unit = { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -612,7 +602,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("in-memory LRU for partitions of same RDD") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -631,7 +621,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("in-memory LRU for partitions of multiple RDDs") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) @@ -654,7 +644,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("on-disk storage") { _conf => - store = makeBlockManager(1200, testConf = Some(_conf)) + val store = makeBlockManager(1200, testConf = Some(_conf)) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -694,7 +684,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE storageLevel: StorageLevel, getAsBytes: Boolean, testConf: SparkConf): Unit = { - store = makeBlockManager(12000, testConf = Some(testConf)) + val store = makeBlockManager(12000, testConf = Some(testConf)) val accessMethod = if (getAsBytes) store.getLocalBytesAndReleaseLock else store.getSingleAndReleaseLock val a1 = new Array[Byte](4000) @@ -723,7 +713,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("LRU with mixed storage levels") { _conf => - store = makeBlockManager(12000, testConf = Some(_conf)) + val store = makeBlockManager(12000, testConf = Some(_conf)) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -745,7 +735,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("in-memory LRU with streams") { _conf => - store = makeBlockManager(12000, testConf = Some(_conf)) + val store = makeBlockManager(12000, testConf = Some(_conf)) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) @@ -773,7 +763,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("LRU with mixed storage levels and streams") { _conf => - store = makeBlockManager(12000, testConf = Some(_conf)) + val store = makeBlockManager(12000, testConf = Some(_conf)) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) @@ -826,7 +816,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("overly large block") { - store = makeBlockManager(5000) + val store = makeBlockManager(5000) store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.getSingleAndReleaseLock("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) @@ -836,65 +826,58 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("block compression") { try { - conf.set("spark.shuffle.compress", "true") - store = makeBlockManager(20000, "exec1") + conf.set(SHUFFLE_COMPRESS, true) + var store = makeBlockManager(20000, "exec1") store.putSingle( ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.shuffle.compress", "false") + conf.set(SHUFFLE_COMPRESS, false) store = makeBlockManager(20000, "exec2") store.putSingle( ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.broadcast.compress", "true") + conf.set(BROADCAST_COMPRESS, true) store = makeBlockManager(20000, "exec3") store.putSingle( BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.broadcast.compress", "false") + conf.set(BROADCAST_COMPRESS, false) store = makeBlockManager(20000, "exec4") store.putSingle( BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.rdd.compress", "true") + conf.set(RDD_COMPRESS, true) store = makeBlockManager(20000, "exec5") store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 1000, "rdd_0_0 was not compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.rdd.compress", "false") + conf.set(RDD_COMPRESS, false) store = makeBlockManager(20000, "exec6") store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 10000, "rdd_0_0 was compressed") - store.stop() - store = null + stopBlockManager(store) // Check that any other block types are also kept uncompressed store = makeBlockManager(20000, "exec7") store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed") - store.stop() - store = null + stopBlockManager(store) } finally { - System.clearProperty("spark.shuffle.compress") - System.clearProperty("spark.broadcast.compress") - System.clearProperty("spark.rdd.compress") + System.clearProperty(SHUFFLE_COMPRESS.key) + System.clearProperty(BROADCAST_COMPRESS.key) + System.clearProperty(RDD_COMPRESS.key) } } @@ -904,7 +887,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) val memoryManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) - store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, + val store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, serializerManager, conf, memoryManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) memoryManager.setMemoryStore(store.memoryStore) @@ -926,7 +909,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("turn off updated block statuses") { val conf = new SparkConf() conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, false) - store = makeBlockManager(12000, testConf = Some(conf)) + val store = makeBlockManager(12000, testConf = Some(conf)) store.registerTask(0) val list = List.fill(2)(new Array[Byte](2000)) @@ -954,7 +937,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("updated block statuses") { val conf = new SparkConf() conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, true) - store = makeBlockManager(12000, testConf = Some(conf)) + val store = makeBlockManager(12000, testConf = Some(conf)) store.registerTask(0) val list = List.fill(2)(new Array[Byte](2000)) val bigList = List.fill(8)(new Array[Byte](2000)) @@ -1052,7 +1035,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("query block statuses") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. @@ -1097,7 +1080,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("get matching blocks") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val list = List.fill(2)(new Array[Byte](100)) // insert some blocks @@ -1141,7 +1124,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. @@ -1155,7 +1138,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("safely unroll blocks through putIterator (disk)") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val memoryStore = store.memoryStore val diskStore = store.diskStore val smallList = List.fill(40)(new Array[Byte](100)) @@ -1194,7 +1177,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("read-locked blocks cannot be evicted from memory") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val arr = new Array[Byte](4000) // First store a1 and a2, both in memory, and a3, on disk only store.putSingle("a1", arr, StorageLevel.MEMORY_ONLY_SER) @@ -1220,7 +1203,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE private def testReadWithLossOfOnDiskFiles( storageLevel: StorageLevel, readMethod: BlockManager => Option[_]): Unit = { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) assert(store.putSingle("blockId", new Array[Byte](4000), storageLevel)) assert(store.getStatus("blockId").isDefined) // Directly delete all files from the disk store, triggering failures when reading blocks: @@ -1260,7 +1243,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") { val mockBlockTransferService = new MockBlockTransferService(conf.getInt("spark.block.failures.beforeLocationRefresh", 5)) - store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + val store = + makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getRemoteBytes("item").isEmpty) } @@ -1280,7 +1264,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn( blockManagerIds) - store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, + val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, transferService = Option(mockBlockTransferService)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] @@ -1301,8 +1285,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE throw new InterruptedException("Intentional interrupt") } } - store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) - store2 = makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) + val store = + makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + val store2 = + makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) intercept[InterruptedException] { store.putSingle("item", "value", StorageLevel.MEMORY_ONLY_2, tellMaster = true) } @@ -1312,8 +1298,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-17484: master block locations are updated following an invalid remote block fetch") { - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") + val store = makeBlockManager(8000, "executor1") + val store2 = makeBlockManager(8000, "executor2") store.putSingle("item", "value", StorageLevel.MEMORY_ONLY, tellMaster = true) assert(master.getLocations("item").nonEmpty) store.removeBlock("item", tellMaster = false) @@ -1410,7 +1396,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE Option(BlockLocationsAndStatus(blockLocations, blockStatus))) when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockLocations) - store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, + val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, transferService = Option(mockBlockTransferService)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] diff --git a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala index 6f60b08088cd..97b9c973e97f 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala @@ -39,7 +39,7 @@ class DiskStoreSuite extends SparkFunSuite { // It will cause error when we tried to re-open the filestore and the // memory-mapped byte buffer tot he file has not been GC on Windows. assume(!Utils.isWindows) - val confKey = "spark.storage.memoryMapThreshold" + val confKey = config.STORAGE_MEMORY_MAP_THRESHOLD.key // Create a non-trivial (not all zeros) byte array val bytes = Array.tabulate[Byte](1000)(_.toByte) diff --git a/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala index 42828506895a..fc16fe362882 100644 --- a/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage import org.apache.spark._ - +import org.apache.spark.internal.config._ class FlatmapIteratorSuite extends SparkFunSuite with LocalSparkContext { /* Tests the ability of Spark to deal with user provided iterators from flatMap @@ -55,7 +55,7 @@ class FlatmapIteratorSuite extends SparkFunSuite with LocalSparkContext { test("Serializer Reset") { val sconf = new SparkConf().setMaster("local").setAppName("serializer_reset_test") - .set("spark.serializer.objectStreamReset", "10") + .set(SERIALIZER_OBJECT_STREAM_RESET, 10) sc = new SparkContext(sconf) val expand_size = 500 val data = sc.parallelize(Seq(1, 2)). 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 7cdcd0fea2ed..5595ce4d8404 100644 --- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala @@ -26,6 +26,7 @@ import scala.reflect.ClassTag import org.scalatest._ import org.apache.spark._ +import org.apache.spark.internal.config.STORAGE_UNROLL_MEMORY_THRESHOLD import org.apache.spark.memory.{MemoryMode, UnifiedMemoryManager} import org.apache.spark.serializer.{KryoSerializer, SerializerManager} import org.apache.spark.storage.memory.{BlockEvictionHandler, MemoryStore, PartiallySerializedBlock, PartiallyUnrolledIterator} @@ -39,7 +40,7 @@ class MemoryStoreSuite with ResetSystemProperties { var conf: SparkConf = new SparkConf(false) - .set("spark.storage.unrollMemoryThreshold", "512") + .set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test val serializer = new KryoSerializer(new SparkConf(false).set("spark.kryoserializer.buffer", "1m")) diff --git a/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala index bbd252d7be7e..0bc26adeeb44 100644 --- a/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala @@ -22,6 +22,7 @@ import java.io.{File, FileOutputStream} import org.scalatest.{BeforeAndAfter, Matchers} import org.apache.spark._ +import org.apache.spark.internal.config.STORAGE_REPLICATION_TOPOLOGY_FILE import org.apache.spark.util.Utils class TopologyMapperSuite extends SparkFunSuite @@ -36,7 +37,7 @@ class TopologyMapperSuite extends SparkFunSuite val propsFile = createPropertiesFile(props) val sparkConf = (new SparkConf(false)) - sparkConf.set("spark.storage.replication.topologyFile", propsFile.getAbsolutePath) + sparkConf.set(STORAGE_REPLICATION_TOPOLOGY_FILE, propsFile.getAbsolutePath) val topologyMapper = new FileBasedTopologyMapper(sparkConf) props.foreach {case (host, topology) => diff --git a/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala b/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala index f46cc293ed27..098d012eed88 100644 --- a/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala @@ -28,7 +28,7 @@ import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{mock, times, verify, when} import org.apache.spark._ -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ class HttpSecurityFilterSuite extends SparkFunSuite { @@ -92,9 +92,9 @@ class HttpSecurityFilterSuite extends SparkFunSuite { test("perform access control") { val conf = new SparkConf(false) - .set("spark.ui.acls.enable", "true") - .set("spark.admin.acls", "admin") - .set("spark.ui.view.acls", "alice") + .set(ACLS_ENABLE, true) + .set(ADMIN_ACLS, Seq("admin")) + .set(UI_VIEW_ACLS, Seq("alice")) val secMgr = new SecurityManager(conf) val req = mockEmptyRequest() @@ -123,7 +123,7 @@ class HttpSecurityFilterSuite extends SparkFunSuite { test("set security-related headers") { val conf = new SparkConf(false) - .set("spark.ui.allowFramingFrom", "example.com") + .set(UI_ALLOW_FRAMING_FROM, "example.com") .set(UI_X_XSS_PROTECTION, "xssProtection") .set(UI_X_CONTENT_TYPE_OPTIONS, true) .set(UI_STRICT_TRANSPORT_SECURITY, "tsec") diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index b04b065f9ecb..b184b74bf3cb 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -39,8 +39,9 @@ import org.apache.spark._ import org.apache.spark.LocalSparkContext._ import org.apache.spark.api.java.StorageLevels import org.apache.spark.deploy.history.HistoryServerSuite -import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE +import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Status._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.status.api.v1.{JacksonMessageWriter, RDDDataDistribution, StageStatus} @@ -103,9 +104,9 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.ui.enabled", "true") - .set("spark.ui.port", "0") - .set("spark.ui.killEnabled", killEnabled.toString) + .set(UI_ENABLED, true) + .set(UI_PORT, 0) + .set(UI_KILL_ENABLED, killEnabled) .set(MEMORY_OFFHEAP_SIZE.key, "64m") val sc = new SparkContext(conf) assert(sc.ui.isDefined) @@ -531,8 +532,8 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.ui.enabled", "true") - .set("spark.ui.port", "0") + .set(UI_ENABLED, true) + .set(UI_PORT, 0) .set(MAX_RETAINED_STAGES, 3) .set(MAX_RETAINED_JOBS, 2) .set(ASYNC_TRACKING_ENABLED, false) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index eaa8f28ae062..1bd7aed7920b 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -32,6 +32,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.util.Utils class UISuite extends SparkFunSuite { @@ -44,7 +45,7 @@ class UISuite extends SparkFunSuite { val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.ui.enabled", "true") + .set(UI_ENABLED, true) val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc 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 b88f25726fc4..c63f04db184d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -66,6 +66,7 @@ class JsonProtocolSuite extends SparkFunSuite { val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), "Spark Properties" -> Seq(("Job throughput", "80000 jobs/s, regardless of job type")), + "Hadoop Properties" -> Seq(("hadoop.tmp.dir", "/usr/local/hadoop/tmp")), "System Properties" -> Seq(("Username", "guest"), ("Password", "guest")), "Classpath Entries" -> Seq(("Super library", "/tmp/super_library")) )) @@ -1761,6 +1762,9 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Spark Properties": { | "Job throughput": "80000 jobs/s, regardless of job type" | }, + | "Hadoop Properties": { + | "hadoop.tmp.dir": "/usr/local/hadoop/tmp" + | }, | "System Properties": { | "Username": "guest", | "Password": "guest" diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index d3f94fbe05d7..7aca0ad3d5c3 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TaskContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.SparkListener @@ -829,7 +830,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { test("isDynamicAllocationEnabled") { val conf = new SparkConf() conf.set("spark.master", "yarn") - conf.set("spark.submit.deployMode", "client") + conf.set(SUBMIT_DEPLOY_MODE, "client") assert(Utils.isDynamicAllocationEnabled(conf) === false) assert(Utils.isDynamicAllocationEnabled( conf.set("spark.dynamicAllocation.enabled", "false")) === false) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 1e0399809ba8..43abb56254dd 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -54,13 +54,13 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite val conf = new SparkConf(loadDefaults) // Make the Java serializer write a reset instruction (TC_RESET) after each object to test // for a bug we had with bytes written past the last object in a batch (SPARK-2792) - conf.set("spark.serializer.objectStreamReset", "1") - conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") - conf.set("spark.shuffle.spill.compress", codec.isDefined.toString) - conf.set("spark.shuffle.compress", codec.isDefined.toString) - codec.foreach { c => conf.set("spark.io.compression.codec", c) } + conf.set(SERIALIZER_OBJECT_STREAM_RESET, 1) + conf.set(SERIALIZER, "org.apache.spark.serializer.JavaSerializer") + conf.set(SHUFFLE_SPILL_COMPRESS, codec.isDefined) + conf.set(SHUFFLE_COMPRESS, codec.isDefined) + codec.foreach { c => conf.set(IO_COMPRESSION_CODEC, c) } // Ensure that we actually have multiple batches per spill file - conf.set("spark.shuffle.spill.batchSize", "10") + conf.set(SHUFFLE_SPILL_BATCH_SIZE, 10L) conf } @@ -253,7 +253,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite private def testSimpleSpilling(codec: Option[String] = None, encrypt: Boolean = false): Unit = { val size = 1000 val conf = createSparkConf(loadDefaults = true, codec) // Load defaults for Spark home - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 4).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 4) conf.set(IO_ENCRYPTION_ENABLED, encrypt) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) @@ -297,7 +297,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite test("ExternalAppendOnlyMap shouldn't fail when forced to spill before calling its iterator") { val size = 1000 val conf = createSparkConf(loadDefaults = true) - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) val map = createExternalMap[String] val consumer = createExternalMap[String] @@ -308,7 +308,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite test("spilling with hash collisions") { val size = 1000 val conf = createSparkConf(loadDefaults = true) - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) val map = createExternalMap[String] @@ -359,7 +359,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite test("spilling with many hash collisions") { val size = 1000 val conf = createSparkConf(loadDefaults = true) - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) val context = MemoryTestingUtils.fakeTaskContext(sc.env) val map = @@ -388,7 +388,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite test("spilling with hash collisions using the Int.MaxValue key") { val size = 1000 val conf = createSparkConf(loadDefaults = true) - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) val map = createExternalMap[Int] @@ -407,7 +407,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite test("spilling with null keys and values") { val size = 1000 val conf = createSparkConf(loadDefaults = true) - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) val map = createExternalMap[Int] @@ -533,7 +533,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite test("SPARK-22713 external aggregation updates peak execution memory") { val spillThreshold = 1000 val conf = createSparkConf(loadDefaults = false) - .set("spark.shuffle.spill.numElementsForceSpillThreshold", spillThreshold.toString) + .set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, spillThreshold) sc = new SparkContext("local", "test", conf) // No spilling AccumulatorSuite.verifyPeakExecutionMemorySet(sc, "external map without spilling") { @@ -553,7 +553,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite val conf = createSparkConf(loadDefaults = false) .set("spark.memory.storageFraction", "0.999") .set(TEST_MEMORY, 471859200L) - .set("spark.shuffle.sort.bypassMergeThreshold", "0") + .set(SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD, 0) sc = new SparkContext("local", "test", conf) val N = 200000 sc.parallelize(1 to N, 2) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 14148e0e67fa..d6c1562a7c59 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark._ +import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests.TEST_MEMORY import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} @@ -123,7 +124,7 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { test("spilling with hash collisions") { val size = 1000 val conf = createSparkConf(loadDefaults = true, kryo = false) - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) val context = MemoryTestingUtils.fakeTaskContext(sc.env) @@ -184,7 +185,7 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { test("spilling with many hash collisions") { val size = 1000 val conf = createSparkConf(loadDefaults = true, kryo = false) - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) val context = MemoryTestingUtils.fakeTaskContext(sc.env) val agg = new Aggregator[FixedHashObject, Int, Int](_ => 1, _ + _, _ + _) @@ -207,7 +208,7 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { test("spilling with hash collisions using the Int.MaxValue key") { val size = 1000 val conf = createSparkConf(loadDefaults = true, kryo = false) - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) val context = MemoryTestingUtils.fakeTaskContext(sc.env) @@ -233,7 +234,7 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { test("spilling with null keys and values") { val size = 1000 val conf = createSparkConf(loadDefaults = true, kryo = false) - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) val context = MemoryTestingUtils.fakeTaskContext(sc.env) @@ -268,17 +269,17 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { private def createSparkConf(loadDefaults: Boolean, kryo: Boolean): SparkConf = { val conf = new SparkConf(loadDefaults) if (kryo) { - conf.set("spark.serializer", classOf[KryoSerializer].getName) + conf.set(SERIALIZER, classOf[KryoSerializer].getName) } else { // Make the Java serializer write a reset instruction (TC_RESET) after each object to test // for a bug we had with bytes written past the last object in a batch (SPARK-2792) - conf.set("spark.serializer.objectStreamReset", "1") - conf.set("spark.serializer", classOf[JavaSerializer].getName) + conf.set(SERIALIZER_OBJECT_STREAM_RESET, 1) + conf.set(SERIALIZER, classOf[JavaSerializer].getName) } - conf.set("spark.shuffle.sort.bypassMergeThreshold", "0") + conf.set(SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD, 0) // Ensure that we actually have multiple batches per spill file - conf.set("spark.shuffle.spill.batchSize", "10") - conf.set("spark.shuffle.spill.initialMemoryThreshold", "512") + conf.set(SHUFFLE_SPILL_BATCH_SIZE, 10L) + conf.set(SHUFFLE_SPILL_INITIAL_MEM_THRESHOLD, 512L) conf } @@ -301,7 +302,7 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { * =========================================== */ private def emptyDataStream(conf: SparkConf) { - conf.set("spark.shuffle.manager", "sort") + conf.set(SHUFFLE_MANAGER, "sort") sc = new SparkContext("local", "test", conf) val context = MemoryTestingUtils.fakeTaskContext(sc.env) @@ -334,7 +335,7 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { } private def fewElementsPerPartition(conf: SparkConf) { - conf.set("spark.shuffle.manager", "sort") + conf.set(SHUFFLE_MANAGER, "sort") sc = new SparkContext("local", "test", conf) val context = MemoryTestingUtils.fakeTaskContext(sc.env) @@ -376,8 +377,8 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { private def emptyPartitionsWithSpilling(conf: SparkConf) { val size = 1000 - conf.set("spark.shuffle.manager", "sort") - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_MANAGER, "sort") + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local", "test", conf) val context = MemoryTestingUtils.fakeTaskContext(sc.env) @@ -401,8 +402,8 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { private def testSpillingInLocalCluster(conf: SparkConf, numReduceTasks: Int) { val size = 5000 - conf.set("spark.shuffle.manager", "sort") - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 4).toString) + conf.set(SHUFFLE_MANAGER, "sort") + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 4) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) assertSpilled(sc, "reduceByKey") { @@ -461,8 +462,8 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { private def cleanupIntermediateFilesInSorter(withFailures: Boolean): Unit = { val size = 1200 val conf = createSparkConf(loadDefaults = false, kryo = false) - conf.set("spark.shuffle.manager", "sort") - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 4).toString) + conf.set(SHUFFLE_MANAGER, "sort") + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 4) sc = new SparkContext("local", "test", conf) val diskBlockManager = sc.env.blockManager.diskBlockManager val ord = implicitly[Ordering[Int]] @@ -490,8 +491,8 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { private def cleanupIntermediateFilesInShuffle(withFailures: Boolean): Unit = { val size = 1200 val conf = createSparkConf(loadDefaults = false, kryo = false) - conf.set("spark.shuffle.manager", "sort") - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 4).toString) + conf.set(SHUFFLE_MANAGER, "sort") + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 4) sc = new SparkContext("local", "test", conf) val diskBlockManager = sc.env.blockManager.diskBlockManager val data = sc.parallelize(0 until size, 2).map { i => @@ -526,9 +527,9 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { withSpilling: Boolean) { val size = 1000 if (withSpilling) { - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) } - conf.set("spark.shuffle.manager", "sort") + conf.set(SHUFFLE_MANAGER, "sort") sc = new SparkContext("local", "test", conf) val agg = if (withPartialAgg) { @@ -560,8 +561,8 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { private def sortWithoutBreakingSortingContracts(conf: SparkConf) { val size = 100000 val conf = createSparkConf(loadDefaults = true, kryo = false) - conf.set("spark.shuffle.manager", "sort") - conf.set("spark.shuffle.spill.numElementsForceSpillThreshold", (size / 2).toString) + conf.set(SHUFFLE_MANAGER, "sort") + conf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, size / 2) sc = new SparkContext("local-cluster[1,1,1024]", "test", conf) // Using wrongOrdering to show integer overflow introduced exception. @@ -618,8 +619,8 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { test("sorting updates peak execution memory") { val spillThreshold = 1000 val conf = createSparkConf(loadDefaults = false, kryo = false) - .set("spark.shuffle.manager", "sort") - .set("spark.shuffle.spill.numElementsForceSpillThreshold", spillThreshold.toString) + .set(SHUFFLE_MANAGER, "sort") + .set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, spillThreshold) sc = new SparkContext("local", "test", conf) // Avoid aggregating here to make sure we're not also using ExternalAppendOnlyMap // No spilling @@ -640,7 +641,7 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { val conf = createSparkConf(loadDefaults = false, kryo = false) .set("spark.memory.storageFraction", "0.999") .set(TEST_MEMORY, 471859200L) - .set("spark.shuffle.sort.bypassMergeThreshold", "0") + .set(SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD, 0) sc = new SparkContext("local", "test", conf) val N = 200000 val p = new org.apache.spark.HashPartitioner(2) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 2fdb5c8dd38a..3b5f9ef2afe8 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -176,10 +176,14 @@ if [[ "$1" == "package" ]]; then # Source and binary tarballs echo "Packaging release source tarballs" cp -r spark spark-$SPARK_VERSION - # For source release, exclude copy of binary license/notice - rm spark-$SPARK_VERSION/LICENSE-binary - rm spark-$SPARK_VERSION/NOTICE-binary - rm -r spark-$SPARK_VERSION/licenses-binary + + # For source release in v2.4+, exclude copy of binary license/notice + if [[ $SPARK_VERSION > "2.4" ]]; then + rm spark-$SPARK_VERSION/LICENSE-binary + rm spark-$SPARK_VERSION/NOTICE-binary + rm -r spark-$SPARK_VERSION/licenses-binary + fi + tar cvzf spark-$SPARK_VERSION.tgz spark-$SPARK_VERSION echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour --output spark-$SPARK_VERSION.tgz.asc \ --detach-sig spark-$SPARK_VERSION.tgz diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index f273b337fdb4..a5a26ae8f535 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -236,7 +236,7 @@ def translate_component(component, commit_hash, warnings): # The returned components are already filtered and translated def find_components(commit, commit_hash): components = re.findall(r"\[\w*\]", commit.lower()) - components = [translate_component(c, commit_hash) + components = [translate_component(c, commit_hash, []) for c in components if c in known_components] return components diff --git a/dev/pip-sanity-check.py b/dev/pip-sanity-check.py index c491005f4971..4171f28684d5 100644 --- a/dev/pip-sanity-check.py +++ b/dev/pip-sanity-check.py @@ -18,7 +18,6 @@ from __future__ import print_function from pyspark.sql import SparkSession -from pyspark.ml.param import Params from pyspark.mllib.linalg import * import sys diff --git a/dev/run-tests.py b/dev/run-tests.py index 27f7527052e2..e1ed2744d78b 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -29,7 +29,7 @@ from sparktestsupport import SPARK_HOME, USER_HOME, ERROR_CODES from sparktestsupport.shellutils import exit_from_command_with_retcode, run_cmd, rm_r, which -from sparktestsupport.toposort import toposort_flatten, toposort +from sparktestsupport.toposort import toposort_flatten import sparktestsupport.modules as modules @@ -153,30 +153,6 @@ def determine_java_executable(): return java_exe if java_exe else which("java") -JavaVersion = namedtuple('JavaVersion', ['major', 'minor', 'patch']) - - -def determine_java_version(java_exe): - """Given a valid java executable will return its version in named tuple format - with accessors '.major', '.minor', '.patch', '.update'""" - - raw_output = subprocess.check_output([java_exe, "-version"], - stderr=subprocess.STDOUT, - universal_newlines=True) - - raw_output_lines = raw_output.split('\n') - - # find raw version string, eg 'java version "1.8.0_25"' - raw_version_str = next(x for x in raw_output_lines if " version " in x) - - match = re.search(r'(\d+)\.(\d+)\.(\d+)', raw_version_str) - - major = int(match.group(1)) - minor = int(match.group(2)) - patch = int(match.group(3)) - - return JavaVersion(major, minor, patch) - # ------------------------------------------------------------------------------------------------- # Functions for running the other build and test scripts # ------------------------------------------------------------------------------------------------- @@ -443,7 +419,6 @@ def run_python_packaging_tests(): def run_build_tests(): set_title_and_block("Running build tests", "BLOCK_BUILD_TESTS") run_cmd([os.path.join(SPARK_HOME, "dev", "test-dependencies.sh")]) - pass def run_sparkr_tests(): @@ -495,8 +470,6 @@ def main(): " install one and retry.") sys.exit(2) - java_version = determine_java_version(java_exe) - # install SparkR if which("R"): run_cmd([os.path.join(SPARK_HOME, "R", "install-dev.sh")]) diff --git a/docs/configuration.md b/docs/configuration.md index 3c383eed04d8..7d3bbf93ae96 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -169,7 +169,7 @@ of the most common options to set are: The amount of off-heap memory to be allocated per driver in cluster mode, in MiB unless otherwise specified. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size (typically 6-10%). - This option is currently supported on YARN and Kubernetes. + This option is currently supported on YARN, Mesos and Kubernetes. diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md index c65ecdcb67ee..896d95bd4884 100644 --- a/docs/mllib-evaluation-metrics.md +++ b/docs/mllib-evaluation-metrics.md @@ -413,13 +413,13 @@ A ranking system usually deals with a set of $M$ users $$U = \left\{u_0, u_1, ..., u_{M-1}\right\}$$ -Each user ($u_i$) having a set of $N$ ground truth relevant documents +Each user ($u_i$) having a set of $N_i$ ground truth relevant documents -$$D_i = \left\{d_0, d_1, ..., d_{N-1}\right\}$$ +$$D_i = \left\{d_0, d_1, ..., d_{N_i-1}\right\}$$ -And a list of $Q$ recommended documents, in order of decreasing relevance +And a list of $Q_i$ recommended documents, in order of decreasing relevance -$$R_i = \left[r_0, r_1, ..., r_{Q-1}\right]$$ +$$R_i = \left[r_0, r_1, ..., r_{Q_i-1}\right]$$ The goal of the ranking system is to produce the most relevant set of documents for each user. The relevance of the sets and the effectiveness of the algorithms can be measured using the metrics listed below. @@ -439,10 +439,10 @@ $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{ Precision at k - $p(k)=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{k} \sum_{j=0}^{\text{min}(\left|D\right|, k) - 1} rel_{D_i}(R_i(j))}$ + $p(k)=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{k} \sum_{j=0}^{\text{min}(Q_i, k) - 1} rel_{D_i}(R_i(j))}$ - Precision at k is a measure of + Precision at k is a measure of how many of the first k recommended documents are in the set of true relevant documents averaged across all users. In this metric, the order of the recommendations is not taken into account. @@ -450,10 +450,10 @@ $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{ Mean Average Precision - $MAP=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{\left|D_i\right|} \sum_{j=0}^{Q-1} \frac{rel_{D_i}(R_i(j))}{j + 1}}$ + $MAP=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{N_i} \sum_{j=0}^{Q_i-1} \frac{rel_{D_i}(R_i(j))}{j + 1}}$ - MAP is a measure of how + MAP is a measure of how many of the recommended documents are in the set of true relevant documents, where the order of the recommendations is taken into account (i.e. penalty for highly relevant documents is higher). @@ -462,10 +462,10 @@ $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{ Normalized Discounted Cumulative Gain $NDCG(k)=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{IDCG(D_i, k)}\sum_{j=0}^{n-1} - \frac{rel_{D_i}(R_i(j))}{\text{ln}(j+2)}} \\ + \frac{rel_{D_i}(R_i(j))}{\text{log}(j+2)}} \\ \text{Where} \\ - \hspace{5 mm} n = \text{min}\left(\text{max}\left(|R_i|,|D_i|\right),k\right) \\ - \hspace{5 mm} IDCG(D, k) = \sum_{j=0}^{\text{min}(\left|D\right|, k) - 1} \frac{1}{\text{ln}(j+2)}$ + \hspace{5 mm} n = \text{min}\left(\text{max}\left(Q_i, N_i\right),k\right) \\ + \hspace{5 mm} IDCG(D, k) = \sum_{j=0}^{\text{min}(\left|D\right|, k) - 1} \frac{1}{\text{log}(j+2)}$ NDCG at k is a diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index a2d782e782ae..5d3d4c6ece39 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -17,8 +17,6 @@ displayTitle: Spark SQL Upgrading Guide - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. - - In Spark version 2.4 and earlier, the `from_json` function produces `null`s for JSON strings and JSON datasource skips the same independently of its mode if there is no valid root JSON token in its input (` ` for example). Since Spark 3.0, such input is treated as a bad record and handled according to specified mode. For example, in the `PERMISSIVE` mode the ` ` input is converted to `Row(null, null)` if specified schema is `key STRING, value INT`. - - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set. - In Spark version 2.4 and earlier, users can create map values with map type key via built-in function like `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can still read map values with map type key from data source or Java/Scala collections, though they are not very useful. @@ -33,13 +31,13 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. - - Since Spark 3.0, CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpose with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. + - Since Spark 3.0, CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpose with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. - - Since Spark 3.0, the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions use java.time API for parsing and formatting dates/timestamps from/to strings by using ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html) based on Proleptic Gregorian calendar. In Spark version 2.4 and earlier, java.text.SimpleDateFormat and java.util.GregorianCalendar (hybrid calendar that supports both the Julian and Gregorian calendar systems, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html) is used for the same purpose. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. To switch back to the implementation used in Spark 2.4 and earlier, set `spark.sql.legacy.timeParser.enabled` to `true`. + - Since Spark 3.0, the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions use java.time API for parsing and formatting dates/timestamps from/to strings by using ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html) based on Proleptic Gregorian calendar. In Spark version 2.4 and earlier, java.text.SimpleDateFormat and java.util.GregorianCalendar (hybrid calendar that supports both the Julian and Gregorian calendar systems, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html) is used for the same purpose. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. - Since Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inferring. diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 3d64ec4cb55f..c19aa5c504b0 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -379,7 +379,25 @@ The following configurations are optional: string spark-kafka-source streaming and batch - Prefix of consumer group identifiers (`group.id`) that are generated by structured streaming queries + Prefix of consumer group identifiers (`group.id`) that are generated by structured streaming + queries. If "kafka.group.id" is set, this option will be ignored. + + + kafka.group.id + string + none + streaming and batch + The Kafka group id to use in Kafka consumer while reading from Kafka. Use this with caution. + By default, each query generates a unique group id for reading data. This ensures that each Kafka + source has its own consumer group that does not face interference from any other consumer, and + therefore can read all of the partitions of its subscribed topics. In some scenarios (for example, + Kafka group-based authorization), you may want to use a specific authorized group id to read data. + You can optionally set the group id. However, do this with extreme caution as it can cause + unexpected behavior. Concurrently running queries (both, batch and streaming) or sources with the + same group id are likely interfere with each other causing each query to read only part of the + data. This may also occur when queries are started/restarted in quick succession. To minimize such + issues, set the Kafka consumer session timeout (by setting option "kafka.session.timeout.ms") to + be very small. When this is set, option "groupIdPrefix" will be ignored. @@ -592,8 +610,9 @@ for parameters related to writing data. Note that the following Kafka params cannot be set and the Kafka source or sink will throw an exception: - **group.id**: Kafka source will create a unique group id for each query automatically. The user can -set the prefix of the automatically generated group.id's via the optional source option `groupIdPrefix`, default value -is "spark-kafka-source". +set the prefix of the automatically generated group.id's via the optional source option `groupIdPrefix`, +default value is "spark-kafka-source". You can also set "kafka.group.id" to force Spark to use a special +group id, however, please read warnings for this option and use it with caution. - **auto.offset.reset**: Set the source option `startingOffsets` to specify where to start instead. Structured Streaming manages which offsets are consumed internally, rather than rely on the kafka Consumer to do it. This will ensure that no data is missed when new diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java index 0707db8d3e83..7ee3c0a3e192 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java @@ -50,7 +50,7 @@ public static void main(String[] args) { // Empty categoricalFeaturesInfo indicates all features are continuous. int numClasses = 2; Map categoricalFeaturesInfo = new HashMap<>(); - Integer numTrees = 3; // Use more in practice. + int numTrees = 3; // Use more in practice. String featureSubsetStrategy = "auto"; // Let the algorithm choose. String impurity = "gini"; int maxDepth = 5; diff --git a/examples/src/main/python/mllib/bisecting_k_means_example.py b/examples/src/main/python/mllib/bisecting_k_means_example.py index 31f3e72d7ff1..36e36fc6897f 100644 --- a/examples/src/main/python/mllib/bisecting_k_means_example.py +++ b/examples/src/main/python/mllib/bisecting_k_means_example.py @@ -23,7 +23,7 @@ from pyspark import SparkContext # $example on$ -from pyspark.mllib.clustering import BisectingKMeans, BisectingKMeansModel +from pyspark.mllib.clustering import BisectingKMeans # $example off$ if __name__ == "__main__": diff --git a/examples/src/main/python/mllib/isotonic_regression_example.py b/examples/src/main/python/mllib/isotonic_regression_example.py index 33d618ab48ea..f5322d79c45b 100644 --- a/examples/src/main/python/mllib/isotonic_regression_example.py +++ b/examples/src/main/python/mllib/isotonic_regression_example.py @@ -23,7 +23,7 @@ from pyspark import SparkContext # $example on$ import math -from pyspark.mllib.regression import LabeledPoint, IsotonicRegression, IsotonicRegressionModel +from pyspark.mllib.regression import IsotonicRegression, IsotonicRegressionModel from pyspark.mllib.util import MLUtils # $example off$ diff --git a/examples/src/main/python/mllib/multi_class_metrics_example.py b/examples/src/main/python/mllib/multi_class_metrics_example.py index 7dc5fb4f9127..03a564e75be9 100644 --- a/examples/src/main/python/mllib/multi_class_metrics_example.py +++ b/examples/src/main/python/mllib/multi_class_metrics_example.py @@ -45,9 +45,9 @@ metrics = MulticlassMetrics(predictionAndLabels) # Overall statistics - precision = metrics.precision() - recall = metrics.recall() - f1Score = metrics.fMeasure() + precision = metrics.precision(1.0) + recall = metrics.recall(1.0) + f1Score = metrics.fMeasure(1.0) print("Summary Stats") print("Precision = %s" % precision) print("Recall = %s" % recall) diff --git a/examples/src/main/python/mllib/ranking_metrics_example.py b/examples/src/main/python/mllib/ranking_metrics_example.py index 21333deded35..0913bb34cf9d 100644 --- a/examples/src/main/python/mllib/ranking_metrics_example.py +++ b/examples/src/main/python/mllib/ranking_metrics_example.py @@ -17,7 +17,7 @@ # $example on$ from pyspark.mllib.recommendation import ALS, Rating -from pyspark.mllib.evaluation import RegressionMetrics, RankingMetrics +from pyspark.mllib.evaluation import RegressionMetrics # $example off$ from pyspark import SparkContext diff --git a/examples/src/main/python/mllib/standard_scaler_example.py b/examples/src/main/python/mllib/standard_scaler_example.py index 442094e1bf36..11ed34427dfe 100644 --- a/examples/src/main/python/mllib/standard_scaler_example.py +++ b/examples/src/main/python/mllib/standard_scaler_example.py @@ -19,7 +19,7 @@ from pyspark import SparkContext # $example on$ -from pyspark.mllib.feature import StandardScaler, StandardScalerModel +from pyspark.mllib.feature import StandardScaler from pyspark.mllib.linalg import Vectors from pyspark.mllib.util import MLUtils # $example off$ diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py index 33fc2dfbeefa..e96a8af71adc 100644 --- a/examples/src/main/python/sql/hive.py +++ b/examples/src/main/python/sql/hive.py @@ -23,7 +23,7 @@ from __future__ import print_function # $example on:spark_hive$ -from os.path import expanduser, join, abspath +from os.path import join, abspath from pyspark.sql import SparkSession from pyspark.sql import Row diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index be32cb89f488..e8d5b468df63 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -46,14 +46,15 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { conn.prepareStatement("CREATE TABLE bar (c0 text, c1 integer, c2 double precision, c3 bigint, " + "c4 bit(1), c5 bit(10), c6 bytea, c7 boolean, c8 inet, c9 cidr, " + "c10 integer[], c11 text[], c12 real[], c13 numeric(2,2)[], c14 enum_type, " - + "c15 float4, c16 smallint)").executeUpdate() + + "c15 float4, c16 smallint, c17 numeric[])").executeUpdate() conn.prepareStatement("INSERT INTO bar VALUES ('hello', 42, 1.25, 123456789012345, B'0', " + "B'1000100101', E'\\\\xDEADBEEF', true, '172.16.0.42', '192.168.0.0/16', " - + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}', '{0.11, 0.22}', 'd1', 1.01, 1)""" + + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}', '{0.11, 0.22}', 'd1', 1.01, 1, """ + + "'{111.2222, 333.4444}')" ).executeUpdate() conn.prepareStatement("INSERT INTO bar VALUES (null, null, null, null, null, " + "null, null, null, null, null, " - + "null, null, null, null, null, null, null)" + + "null, null, null, null, null, null, null, null)" ).executeUpdate() conn.prepareStatement("CREATE TABLE ts_with_timezone " + @@ -85,7 +86,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows.length == 2) // Test the types, and values using the first row. val types = rows(0).toSeq.map(x => x.getClass) - assert(types.length == 17) + assert(types.length == 18) assert(classOf[String].isAssignableFrom(types(0))) assert(classOf[java.lang.Integer].isAssignableFrom(types(1))) assert(classOf[java.lang.Double].isAssignableFrom(types(2))) @@ -103,6 +104,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(classOf[String].isAssignableFrom(types(14))) assert(classOf[java.lang.Float].isAssignableFrom(types(15))) assert(classOf[java.lang.Short].isAssignableFrom(types(16))) + assert(classOf[Seq[BigDecimal]].isAssignableFrom(types(17))) assert(rows(0).getString(0).equals("hello")) assert(rows(0).getInt(1) == 42) assert(rows(0).getDouble(2) == 1.25) @@ -123,6 +125,8 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows(0).getString(14) == "d1") assert(rows(0).getFloat(15) == 1.01f) assert(rows(0).getShort(16) == 1) + assert(rows(0).getSeq(17) == + Seq("111.222200000000000000", "333.444400000000000000").map(BigDecimal(_).bigDecimal)) // Test reading null values using the second row. assert(0.until(16).forall(rows(1).isNullAt(_))) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala index 02dfb9ca2b95..f328567c95d2 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import java.util.concurrent.TimeoutException -import org.apache.kafka.clients.consumer.{ConsumerRecord, OffsetOutOfRangeException} +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, OffsetOutOfRangeException} import org.apache.kafka.common.TopicPartition import org.apache.spark.TaskContext @@ -167,7 +167,13 @@ class KafkaContinuousScanConfigBuilder( val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet) if (deletedPartitions.nonEmpty) { - reportDataLoss(s"Some partitions were deleted: $deletedPartitions") + val message = if ( + offsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + } else { + s"$deletedPartitions are gone. Some data may have been missed." + } + reportDataLoss(message) } val startOffsets = newPartitionOffsets ++ diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala index b4f042e93a5d..1c1d26a901b4 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala @@ -22,6 +22,7 @@ import java.io._ import java.nio.charset.StandardCharsets import org.apache.commons.io.IOUtils +import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging @@ -122,7 +123,13 @@ private[kafka010] class KafkaMicroBatchReadSupport( // Find deleted partitions, and report data loss if required val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet) if (deletedPartitions.nonEmpty) { - reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed") + val message = + if (kafkaOffsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + } else { + s"$deletedPartitions are gone. Some data may have been missed." + } + reportDataLoss(message) } // Use the end partitions to calculate offset ranges to ignore partitions that have diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index fc443d22bf5a..14bc6bae6d67 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -46,7 +46,7 @@ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} */ private[kafka010] class KafkaOffsetReader( consumerStrategy: ConsumerStrategy, - driverKafkaParams: ju.Map[String, Object], + val driverKafkaParams: ju.Map[String, Object], readerOptions: Map[String, String], driverGroupIdPrefix: String) extends Logging { /** @@ -82,7 +82,9 @@ private[kafka010] class KafkaOffsetReader( assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) if (_consumer == null) { val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams) - newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId()) + if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) { + newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId()) + } _consumer = consumerStrategy.createConsumer(newKafkaParams) } _consumer diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index d65b3cea632c..da5533409935 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -22,6 +22,7 @@ import java.io._ import java.nio.charset.StandardCharsets import org.apache.commons.io.IOUtils +import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkContext @@ -242,7 +243,12 @@ private[kafka010] class KafkaSource( val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) if (deletedPartitions.nonEmpty) { - reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed") + val message = if (kafkaReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + } else { + s"$deletedPartitions are gone. Some data may have been missed." + } + reportDataLoss(message) } // Use the until partitions to calculate offset ranges to ignore partitions that have diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 5774ee7a1c94..b59f21ab130a 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -335,9 +335,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister // Validate user-specified Kafka options if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { - throw new IllegalArgumentException( - s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " + - s"user-specified consumer groups are not used to track offsets.") + logWarning(CUSTOM_GROUP_ID_ERROR_MESSAGE) + if (caseInsensitiveParams.contains(GROUP_ID_PREFIX)) { + logWarning("Option 'groupIdPrefix' will be ignored as " + + s"option 'kafka.${ConsumerConfig.GROUP_ID_CONFIG}' has been set.") + } } if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { @@ -440,6 +442,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { private[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" private val MIN_PARTITIONS_OPTION_KEY = "minpartitions" + private val GROUP_ID_PREFIX = "groupidprefix" val TOPIC_OPTION_KEY = "topic" @@ -459,7 +462,16 @@ private[kafka010] object KafkaSourceProvider extends Logging { | source option "failOnDataLoss" to "false". """.stripMargin - + val CUSTOM_GROUP_ID_ERROR_MESSAGE = + s"""Kafka option 'kafka.${ConsumerConfig.GROUP_ID_CONFIG}' has been set on this query, it is + | not recommended to set this option. This option is unsafe to use since multiple concurrent + | queries or sources using the same group id will interfere with each other as they are part + | of the same consumer group. Restarted queries may also suffer interference from the + | previous run having the same group id. The user should have only one query per group id, + | and/or set the option 'kafka.session.timeout.ms' to be very small so that the Kafka + | consumers from the previous query are marked dead by the Kafka group coordinator before the + | restarted query starts running. + """.stripMargin private val serClassName = classOf[ByteArraySerializer].getName private val deserClassName = classOf[ByteArrayDeserializer].getName @@ -510,7 +522,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") // So that consumers in executors do not mess with any existing group id - .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") + .setIfUnset(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") // So that consumers in executors does not commit offsets unnecessarily .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") @@ -529,7 +541,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { parameters: Map[String, String], metadataPath: String): String = { val groupIdPrefix = parameters - .getOrElse("groupIdPrefix", "spark-kafka-source") + .getOrElse(GROUP_ID_PREFIX, "spark-kafka-source") s"${groupIdPrefix}-${UUID.randomUUID}-${metadataPath.hashCode}" } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index d4eb52654005..cb453846134e 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -28,6 +28,7 @@ import scala.collection.JavaConverters._ import scala.io.Source import scala.util.Random +import org.apache.kafka.clients.admin.{AdminClient, ConsumerGroupListing} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.PatienceConfiguration.Timeout @@ -629,6 +630,42 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { ) } + test("allow group.id override") { + // Tests code path KafkaSourceProvider.{sourceSchema(.), createSource(.)} + // as well as KafkaOffsetReader.createConsumer(.) + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (1 to 10).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (11 to 20).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(2)) + + val customGroupId = "id-" + Random.nextInt() + val dsKafka = spark + .readStream + .format("kafka") + .option("kafka.group.id", customGroupId) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .load() + .selectExpr("CAST(value AS STRING)") + .as[String] + .map(_.toInt) + + testStream(dsKafka)( + makeSureGetOffsetCalled, + CheckAnswer(1 to 30: _*), + Execute { _ => + val consumerGroups = testUtils.listConsumerGroups() + val validGroups = consumerGroups.valid().get() + val validGroupsId = validGroups.asScala.map(_.groupId()) + assert(validGroupsId.exists(_ === customGroupId), "Valid consumer groups don't " + + s"contain the expected group id - Valid consumer groups: $validGroupsId / " + + s"expected group id: $customGroupId") + } + ) + } + test("ensure stream-stream self-join generates only one offset in log and correct metrics") { val topic = newTopic() testUtils.createTopic(topic, partitions = 2) @@ -1233,7 +1270,6 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("not supported")) } - testUnsupportedConfig("kafka.group.id") testUnsupportedConfig("kafka.auto.offset.reset") testUnsupportedConfig("kafka.enable.auto.commit") testUnsupportedConfig("kafka.interceptor.classes") diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 8cfca56433f5..2cd13a994ee8 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -20,6 +20,9 @@ package org.apache.spark.sql.kafka010 import java.util.Locale import java.util.concurrent.atomic.AtomicInteger +import scala.collection.JavaConverters._ +import scala.util.Random + import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition @@ -239,6 +242,26 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") } + test("allow group.id overriding") { + // Tests code path KafkaSourceProvider.createRelation(.) + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (1 to 10).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (11 to 20).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(2)) + + val customGroupId = "id-" + Random.nextInt() + val df = createDF(topic, withOptions = Map("kafka.group.id" -> customGroupId)) + checkAnswer(df, (1 to 30).map(_.toString).toDF()) + + val consumerGroups = testUtils.listConsumerGroups() + val validGroups = consumerGroups.valid().get() + val validGroupsId = validGroups.asScala.map(_.groupId()) + assert(validGroupsId.exists(_ === customGroupId), "Valid consumer groups don't " + + s"contain the expected group id - Valid consumer groups: $validGroupsId / " + + s"expected group id: $customGroupId") + } + test("read Kafka transactional messages: read_committed") { val topic = newTopic() testUtils.createTopic(topic) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index bf6934be5270..dacfffa86753 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -33,7 +33,7 @@ import kafka.server.{KafkaConfig, KafkaServer} import kafka.server.checkpoints.OffsetCheckpointFile import kafka.utils.ZkUtils import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.clients.admin.{AdminClient, CreatePartitionsOptions, NewPartitions} +import org.apache.kafka.clients.admin.{AdminClient, CreatePartitionsOptions, ListConsumerGroupsResult, NewPartitions} import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ import org.apache.kafka.common.TopicPartition @@ -311,6 +311,10 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L offsets } + def listConsumerGroups(): ListConsumerGroupsResult = { + adminClient.listConsumerGroups() + } + protected def brokerConfiguration: Properties = { val props = new Properties() props.put("broker.id", "0") diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index cc65f78b45c3..e3ee843f6244 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -455,6 +455,7 @@ protected boolean handle(String opt, String value) { conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value); break; case CONF: + checkArgument(value != null, "Missing argument to %s", CONF); String[] setConf = value.split("=", 2); checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value); conf.put(setConf[0], setConf[1]); diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 1b7c15f1f0a8..134d6a9b442a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -88,7 +88,7 @@ private[ml] class WeightedLeastSquares( require(regParam >= 0.0, s"regParam cannot be negative: $regParam") require(elasticNetParam >= 0.0 && elasticNetParam <= 1.0, s"elasticNetParam must be in [0, 1]: $elasticNetParam") - require(maxIter >= 0, s"maxIter must be a positive integer: $maxIter") + require(maxIter > 0, s"maxIter must be a positive integer: $maxIter") require(tol >= 0.0, s"tol must be >= 0, but was set to $tol") /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala index b98aa0534152..4935d1141113 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -138,6 +138,8 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] var dcg = 0.0 var i = 0 while (i < n) { + // Base of the log doesn't matter for calculating NDCG, + // if the relevance value is binary. val gain = 1.0 / math.log(i + 2) if (i < pred.length && labSet.contains(pred(i))) { dcg += gain diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala index eb5f3ca45940..7f892fd509e7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.ml.attribute import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.types._ @@ -225,7 +226,7 @@ class AttributeSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala index cca7399b4b9c..5a7449005839 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.serializer.KryoSerializer class InstanceSuite extends SparkFunSuite{ test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala index 05c7a58ee5ff..63c163590af5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.serializer.KryoSerializer class LabeledPointSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala index f41abe48f2c5..3a44e79291cc 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoSerializer class TreePointSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala index acac171346a8..514fa7f2e1b8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala @@ -22,6 +22,7 @@ import java.io.File import org.scalatest.Suite import org.apache.spark.{DebugFilesystem, SparkConf, SparkContext} +import org.apache.spark.internal.config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK import org.apache.spark.ml.{PredictionModel, Transformer} import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.{DataFrame, Dataset, Encoder, Row} @@ -40,7 +41,7 @@ trait MLTest extends StreamTest with TempDirectory { self: Suite => protected override def sparkConf = { new SparkConf() .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) - .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set(UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) .set(SQLConf.CODEGEN_FALLBACK.key, "false") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index d18cef7e264d..c4bf5b27187f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.clustering import scala.util.Random import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ @@ -316,7 +317,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index f4fa216b8eba..a679fe43414f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.util.Utils class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -109,12 +111,16 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { test("big model load / save") { // backupping old values - val oldBufferConfValue = spark.conf.get("spark.kryoserializer.buffer.max", "64m") - val oldBufferMaxConfValue = spark.conf.get("spark.kryoserializer.buffer", "64k") + val oldBufferConfValue = spark.conf.get(KRYO_SERIALIZER_BUFFER_SIZE.key, "64m") + val oldBufferMaxConfValue = spark.conf.get(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "64k") + val oldSetCommandRejectsSparkCoreConfs = spark.conf.get( + SET_COMMAND_REJECTS_SPARK_CORE_CONFS.key, "true") // setting test values to trigger partitioning - spark.conf.set("spark.kryoserializer.buffer", "50b") - spark.conf.set("spark.kryoserializer.buffer.max", "50b") + + // this is needed to set configurations which are also defined to SparkConf + spark.conf.set(SET_COMMAND_REJECTS_SPARK_CORE_CONFS.key, "false") + spark.conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, "50b") // create a model bigger than 50 Bytes val word2VecMap = Map((0 to 10).map(i => s"$i" -> Array.fill(10)(0.1f)): _*) @@ -137,8 +143,9 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { "that spans over multiple partitions", t) } finally { Utils.deleteRecursively(tempDir) - spark.conf.set("spark.kryoserializer.buffer", oldBufferConfValue) - spark.conf.set("spark.kryoserializer.buffer.max", oldBufferMaxConfValue) + spark.conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, oldBufferConfValue) + spark.conf.set(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, oldBufferMaxConfValue) + spark.conf.set(SET_COMMAND_REJECTS_SPARK_CORE_CONFS.key, oldSetCommandRejectsSparkCoreConfs) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 2c3f84617cfa..b4520d42fedf 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -27,6 +27,7 @@ import org.mockito.Mockito.when import org.scalatest.mockito.MockitoSugar._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.serializer.KryoSerializer @@ -34,7 +35,7 @@ import org.apache.spark.serializer.KryoSerializer class MatricesSuite extends SparkFunSuite { test("kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 217b4a35438f..fee0b02bf8ed 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -25,6 +25,7 @@ import org.json4s.jackson.JsonMethods.{parse => parseJson} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.serializer.KryoSerializer @@ -38,7 +39,7 @@ class VectorsSuite extends SparkFunSuite with Logging { test("kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index c1449ece740d..d3366dcde2ed 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.serializer.KryoSerializer @@ -57,7 +58,7 @@ class LabeledPointSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala index 5b4a2607f0b2..4c88fd3d83a0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.stat.distribution import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.mllib.linalg.{Matrices, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ @@ -83,7 +84,7 @@ class MultivariateGaussianSuite extends SparkFunSuite with MLlibTestSparkContext test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index cf8d9f3c24d0..3e232baaec49 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -227,6 +227,9 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.setActiveContext"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.markPartiallyConstructed"), + // [SPARK-26457] Show hadoop configurations in HistoryServer environment tab + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ApplicationEnvironmentInfo.this"), + // Data Source V2 API changes (problem: Problem) => problem match { case MissingClassProblem(cls) => diff --git a/python/docs/index.rst b/python/docs/index.rst index 421c8de86a3c..0e7b62361802 100644 --- a/python/docs/index.rst +++ b/python/docs/index.rst @@ -37,7 +37,7 @@ Core classes: A Discretized Stream (DStream), the basic abstraction in Spark Streaming. - :class:`pyspark.sql.SQLContext` + :class:`pyspark.sql.SparkSession` Main entry point for DataFrame and SQL functionality. diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 29358b5740e5..69447a6fb561 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -17,7 +17,6 @@ import gc import os -import socket import sys from tempfile import NamedTemporaryFile import threading diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 316fbc8bfda5..94c6f4adab77 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -66,6 +66,10 @@ class SparkContext(object): .. note:: Only one :class:`SparkContext` should be active per JVM. You must `stop()` the active :class:`SparkContext` before creating a new one. + + .. note:: :class:`SparkContext` instance is not supported to share across multiple + processes out of the box, and PySpark does not guarantee multi-processing execution. + Use threads instead for concurrent processing purpose. """ _gateway = None @@ -435,7 +439,6 @@ def stop(self): ' been killed or may also be in a zombie state.', RuntimeWarning ) - pass finally: self._jsc = None if getattr(self, "_accumulatorServer", None): diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index c8c5f801f89b..d8315c63a8fc 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -18,7 +18,6 @@ import atexit import os import sys -import select import signal import shlex import shutil @@ -174,8 +173,7 @@ def local_connect_and_auth(port, auth_secret): errors.append("tried to connect to %s, but an error occured: %s" % (sa, emsg)) sock.close() sock = None - else: - raise Exception("could not open socket: %s" % errors) + raise Exception("could not open socket: %s" % errors) def ensure_callback_server_started(gw): diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 6ddfce95a3d4..89b927814c09 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -23,7 +23,7 @@ from pyspark.ml import Estimator, Model from pyspark.ml.param.shared import * from pyspark.ml.regression import DecisionTreeModel, DecisionTreeRegressionModel, \ - GBTParams, HasVarianceImpurity, RandomForestParams, TreeEnsembleModel, TreeEnsembleParams + GBTParams, HasVarianceImpurity, RandomForestParams, TreeEnsembleModel from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams from pyspark.ml.wrapper import JavaWrapper diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index 734763ebd3fa..ed71fb0c5759 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -18,7 +18,7 @@ from pyspark import keyword_only, since from pyspark.sql import DataFrame from pyspark.ml.util import * -from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams, _jvm +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams from pyspark.ml.param.shared import * __all__ = ["FPGrowth", "FPGrowthModel", "PrefixSpan"] diff --git a/python/pyspark/ml/linalg/__init__.py b/python/pyspark/ml/linalg/__init__.py index 2548fd0f50b3..9da983667be7 100644 --- a/python/pyspark/ml/linalg/__init__.py +++ b/python/pyspark/ml/linalg/__init__.py @@ -270,6 +270,8 @@ class DenseVector(Vector): DenseVector([3.0, 2.0]) >>> u % 2 DenseVector([1.0, 0.0]) + >>> -v + DenseVector([-1.0, -2.0]) """ def __init__(self, ar): if isinstance(ar, bytes): @@ -436,6 +438,9 @@ def __hash__(self): def __getattr__(self, item): return getattr(self.array, item) + def __neg__(self): + return DenseVector(-self.array) + def _delegate(op): def func(self, other): if isinstance(other, DenseVector): @@ -443,7 +448,6 @@ def func(self, other): return DenseVector(getattr(self.array, op)(other)) return func - __neg__ = _delegate("__neg__") __add__ = _delegate("__add__") __sub__ = _delegate("__sub__") __mul__ = _delegate("__mul__") diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 78cb4a670355..9e1f8f88ca1c 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -16,7 +16,6 @@ # import sys -import warnings from pyspark import since, keyword_only from pyspark.ml.param.shared import * diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index e00ed95ef070..d2037be2c64f 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -20,12 +20,10 @@ import warnings import numpy -from numpy import array from pyspark import RDD, since -from pyspark.streaming import DStream from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py -from pyspark.mllib.linalg import DenseVector, SparseVector, _convert_to_vector +from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.regression import ( LabeledPoint, LinearModel, _regression_train_wrapper, StreamingLinearAlgorithm) @@ -126,9 +124,9 @@ class LogisticRegressionModel(LinearClassificationModel): ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(sparse_data), iterations=10) - >>> lrm.predict(array([0.0, 1.0])) + >>> lrm.predict(numpy.array([0.0, 1.0])) 1 - >>> lrm.predict(array([1.0, 0.0])) + >>> lrm.predict(numpy.array([1.0, 0.0])) 0 >>> lrm.predict(SparseVector(2, {1: 1.0})) 1 @@ -138,7 +136,7 @@ class LogisticRegressionModel(LinearClassificationModel): >>> path = tempfile.mkdtemp() >>> lrm.save(sc, path) >>> sameModel = LogisticRegressionModel.load(sc, path) - >>> sameModel.predict(array([0.0, 1.0])) + >>> sameModel.predict(numpy.array([0.0, 1.0])) 1 >>> sameModel.predict(SparseVector(2, {0: 1.0})) 0 @@ -424,7 +422,7 @@ class SVMModel(LinearClassificationModel): >>> svm.predict(sc.parallelize([[1.0]])).collect() [1] >>> svm.clearThreshold() - >>> svm.predict(array([1.0])) + >>> svm.predict(numpy.array([1.0])) 1.44... >>> sparse_data = [ @@ -577,9 +575,9 @@ class NaiveBayesModel(Saveable, Loader): ... LabeledPoint(1.0, [1.0, 0.0]), ... ] >>> model = NaiveBayes.train(sc.parallelize(data)) - >>> model.predict(array([0.0, 1.0])) + >>> model.predict(numpy.array([0.0, 1.0])) 0.0 - >>> model.predict(array([1.0, 0.0])) + >>> model.predict(numpy.array([1.0, 0.0])) 1.0 >>> model.predict(sc.parallelize([[1.0, 0.0]])).collect() [1.0] diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 4f4355ddb60e..58da434fc38a 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -33,7 +33,6 @@ from pyspark.rdd import RDD, ignore_unicode_prefix from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, callJavaFunc, _py2java, _java2py from pyspark.mllib.linalg import SparseVector, _convert_to_vector, DenseVector -from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.stat.distribution import MultivariateGaussian from pyspark.mllib.util import Saveable, Loader, inherit_doc, JavaLoader, JavaSaveable from pyspark.streaming import DStream diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index b171e46871fd..6ca6df672f30 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -16,12 +16,11 @@ # import sys -import warnings from pyspark import since from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc from pyspark.sql import SQLContext -from pyspark.sql.types import StructField, StructType, DoubleType, IntegerType, ArrayType +from pyspark.sql.types import StructField, StructType, DoubleType __all__ = ['BinaryClassificationMetrics', 'RegressionMetrics', 'MulticlassMetrics', 'RankingMetrics'] diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 6d7d4d61db04..b1bcdb9078e3 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -22,8 +22,6 @@ import sys import warnings -import random -import binascii if sys.version >= '3': basestring = str unicode = str diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index 6accb9b4926e..373a141456b2 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -17,11 +17,9 @@ import sys -import numpy -from numpy import array from collections import namedtuple -from pyspark import SparkContext, since +from pyspark import since from pyspark.rdd import ignore_unicode_prefix from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc from pyspark.mllib.util import JavaSaveable, JavaLoader, inherit_doc diff --git a/python/pyspark/mllib/linalg/__init__.py b/python/pyspark/mllib/linalg/__init__.py index 4afd6666400b..94a3e2af4d2d 100644 --- a/python/pyspark/mllib/linalg/__init__.py +++ b/python/pyspark/mllib/linalg/__init__.py @@ -281,6 +281,8 @@ class DenseVector(Vector): DenseVector([3.0, 2.0]) >>> u % 2 DenseVector([1.0, 0.0]) + >>> -v + DenseVector([-1.0, -2.0]) """ def __init__(self, ar): if isinstance(ar, bytes): @@ -480,6 +482,9 @@ def __hash__(self): def __getattr__(self, item): return getattr(self.array, item) + def __neg__(self): + return DenseVector(-self.array) + def _delegate(op): def func(self, other): if isinstance(other, DenseVector): @@ -487,7 +492,6 @@ def func(self, other): return DenseVector(getattr(self.array, op)(other)) return func - __neg__ = _delegate("__neg__") __add__ = _delegate("__add__") __sub__ = _delegate("__sub__") __mul__ = _delegate("__mul__") diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 6be45f51862c..56ee0083abca 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -19,12 +19,11 @@ import warnings import numpy as np -from numpy import array from pyspark import RDD, since from pyspark.streaming.dstream import DStream from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py, inherit_doc -from pyspark.mllib.linalg import SparseVector, Vectors, _convert_to_vector +from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.util import Saveable, Loader __all__ = ['LabeledPoint', 'LinearModel', @@ -168,15 +167,15 @@ class LinearRegressionModel(LinearRegressionModelBase): ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, - ... initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + ... initialWeights=np.array([1.0])) + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, - ... miniBatchFraction=1.0, initialWeights=array([1.0]), regParam=0.1, regType="l2", + ... miniBatchFraction=1.0, initialWeights=np.array([1.0]), regParam=0.1, regType="l2", ... intercept=True, validateData=True) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -305,7 +304,8 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = LassoWithSGD.train(sc.parallelize(data), iterations=10, initialWeights=array([1.0])) + >>> lrm = LassoWithSGD.train( + ... sc.parallelize(data), iterations=10, initialWeights=np.array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 @@ -336,13 +336,13 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, - ... initialWeights=array([1.0])) + ... initialWeights=np.array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True >>> lrm = LassoWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, - ... regParam=0.01, miniBatchFraction=1.0, initialWeights=array([1.0]), intercept=True, + ... regParam=0.01, miniBatchFraction=1.0, initialWeights=np.array([1.0]), intercept=True, ... validateData=True) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True @@ -449,7 +449,7 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(2.0, [3.0]) ... ] >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=10, - ... initialWeights=array([1.0])) + ... initialWeights=np.array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 @@ -480,13 +480,13 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, - ... initialWeights=array([1.0])) + ... initialWeights=np.array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, - ... regParam=0.01, miniBatchFraction=1.0, initialWeights=array([1.0]), intercept=True, + ... regParam=0.01, miniBatchFraction=1.0, initialWeights=np.array([1.0]), intercept=True, ... validateData=True) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True diff --git a/python/pyspark/mllib/tests/test_streaming_algorithms.py b/python/pyspark/mllib/tests/test_streaming_algorithms.py index bf2ad2d267bb..cab30104933a 100644 --- a/python/pyspark/mllib/tests/test_streaming_algorithms.py +++ b/python/pyspark/mllib/tests/test_streaming_algorithms.py @@ -334,7 +334,7 @@ def test_training_and_prediction(self): """Test that the model improves on toy data with no. of batches""" input_batches = [ self.sc.parallelize(self.generateLogisticInput(0, 1.5, 100, 42 + i)) - for i in range(20)] + for i in range(40)] predict_batches = [ b.map(lambda lp: (lp.label, lp.features)) for b in input_batches] diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index b05734ce489d..2d8df461acf9 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -20,7 +20,7 @@ import sys import random -from pyspark import SparkContext, RDD, since +from pyspark import RDD, since from pyspark.mllib.common import callMLlibFunc, inherit_doc, JavaModelWrapper from pyspark.mllib.linalg import _convert_to_vector from pyspark.mllib.regression import LabeledPoint diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 51f20db2927e..0190bf3cc0e3 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -17,7 +17,6 @@ import sys import numpy as np -import warnings if sys.version > '3': xrange = range @@ -420,7 +419,7 @@ def load(cls, sc, path): was saved. :return: model instance """ - raise NotImplemented + raise NotImplementedError @inherit_doc diff --git a/python/pyspark/profiler.py b/python/pyspark/profiler.py index 3c7656ab5758..b9423b760487 100644 --- a/python/pyspark/profiler.py +++ b/python/pyspark/profiler.py @@ -104,11 +104,11 @@ def __init__(self, ctx): def profile(self, func): """ Do profiling on the function `func`""" - raise NotImplemented + raise NotImplementedError def stats(self): """ Return the collected profiling stats (pstats.Stats)""" - raise NotImplemented + raise NotImplementedError def show(self, id): """ Print the profile stats to stdout, id is the RDD id """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index b6e17cab44e9..73969307a5e8 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -25,7 +25,6 @@ import heapq import bisect import random -import socket from subprocess import Popen, PIPE from tempfile import NamedTemporaryFile from threading import Thread @@ -42,8 +41,7 @@ from pyspark.java_gateway import local_connect_and_auth from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ - PickleSerializer, pack_long, AutoBatchedSerializer, write_with_length, \ - UTF8Deserializer + PickleSerializer, pack_long, AutoBatchedSerializer from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_full_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -53,7 +51,7 @@ from pyspark.shuffle import Aggregator, ExternalMerger, \ get_used_memory, ExternalSorter, ExternalGroupBy from pyspark.traceback_utils import SCCallSiteSync -from pyspark.util import fail_on_stopiteration, _exception_message +from pyspark.util import fail_on_stopiteration __all__ = ["RDD"] diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 1938965a7e17..48a49c583f9c 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -537,10 +537,8 @@ def _test(): globs['df'] = rdd.toDF() jsonStrings = [ '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', - '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' - '"field6":[{"field7": "row2"}]}', - '{"field1" : null, "field2": "row3", ' - '"field3":{"field4":33, "field5": []}}' + '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},"field6":[{"field7": "row2"}]}', + '{"field1" : null, "field2": "row3", "field3":{"field4":33, "field5": []}}' ] globs['jsonStrings'] = jsonStrings globs['json'] = sc.parallelize(jsonStrings) diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index fb673f2a385e..3afb88c4351f 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -24,7 +24,7 @@ import unittest from pyspark.sql import Row -from pyspark.sql.functions import UserDefinedFunction +from pyspark.sql.functions import col, UserDefinedFunction from pyspark.sql.types import * from pyspark.sql.types import _array_signed_int_typecode_ctype_mappings, _array_type_mappings, \ _array_unsigned_int_typecode_ctype_mappings, _infer_type, _make_type_verifier, _merge_type @@ -202,6 +202,12 @@ def test_create_dataframe_from_dict_respects_schema(self): df = self.spark.createDataFrame([{'a': 1}], ["b"]) self.assertEqual(df.columns, ['b']) + def test_negative_decimal(self): + df = self.spark.createDataFrame([(1, ), (11, )], ["value"]) + ret = df.select(col("value").cast(DecimalType(1, -1))).collect() + actual = list(map(lambda r: int(r.value), ret)) + self.assertEqual(actual, [0, 10]) + def test_create_dataframe_from_objects(self): data = [MyObject(1, "1"), MyObject(2, "2")] df = self.spark.createDataFrame(data) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 1d24c40e5858..00e90fca615f 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -752,7 +752,7 @@ def __eq__(self, other): for v in [ArrayType, MapType, StructType]) -_FIXED_DECIMAL = re.compile(r"decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)") +_FIXED_DECIMAL = re.compile(r"decimal\(\s*(\d+)\s*,\s*(-?\d+)\s*\)") def _parse_datatype_string(s): @@ -865,6 +865,8 @@ def _parse_datatype_json_string(json_string): >>> complex_maptype = MapType(complex_structtype, ... complex_arraytype, False) >>> check_datatype(complex_maptype) + >>> # Decimal with negative scale. + >>> check_datatype(DecimalType(1,-1)) """ return _parse_datatype_json_value(json.loads(json_string)) @@ -1018,14 +1020,12 @@ def _infer_type(obj): for key, value in obj.items(): if key is not None and value is not None: return MapType(_infer_type(key), _infer_type(value), True) - else: - return MapType(NullType(), NullType(), True) + return MapType(NullType(), NullType(), True) elif isinstance(obj, list): for v in obj: if v is not None: return ArrayType(_infer_type(obj[0]), True) - else: - return ArrayType(NullType(), True) + return ArrayType(NullType(), True) elif isinstance(obj, array): if obj.typecode in _array_type_mappings: return ArrayType(_array_type_mappings[obj.typecode](), False) diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index e1c194b44650..2d84373fb28a 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -17,10 +17,7 @@ from __future__ import print_function -import os -import sys - -from py4j.java_gateway import java_import, JavaObject +from py4j.java_gateway import java_import from pyspark import RDD, SparkConf from pyspark.serializers import NoOpSerializer, UTF8Deserializer, CloudPickleSerializer diff --git a/python/pyspark/streaming/kinesis.py b/python/pyspark/streaming/kinesis.py index b839859c4525..b3348828fdf6 100644 --- a/python/pyspark/streaming/kinesis.py +++ b/python/pyspark/streaming/kinesis.py @@ -15,9 +15,7 @@ # limitations under the License. # -from py4j.protocol import Py4JJavaError - -from pyspark.serializers import PairDeserializer, NoOpSerializer +from pyspark.serializers import NoOpSerializer from pyspark.storagelevel import StorageLevel from pyspark.streaming import DStream diff --git a/python/pyspark/taskcontext.py b/python/pyspark/taskcontext.py index de4b6af23666..dff5e183bdc7 100644 --- a/python/pyspark/taskcontext.py +++ b/python/pyspark/taskcontext.py @@ -16,7 +16,6 @@ # from __future__ import print_function -import socket from pyspark.java_gateway import local_connect_and_auth from pyspark.serializers import write_int, UTF8Deserializer diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index bf007b0c62d8..1e7424ab3a1b 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -28,7 +28,6 @@ import resource except ImportError: has_resource_module = False -import socket import traceback from pyspark.accumulators import _accumulatorRegistry diff --git a/python/run-tests.py b/python/run-tests.py index e45268c13769..7456170ba2d5 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -33,7 +33,6 @@ import Queue else: import queue as Queue -from distutils.version import LooseVersion from multiprocessing import Manager diff --git a/python/setup.py b/python/setup.py index 7da67a4109ed..22f0940db93e 100644 --- a/python/setup.py +++ b/python/setup.py @@ -20,7 +20,7 @@ import glob import os import sys -from setuptools import setup, find_packages +from setuptools import setup from shutil import copyfile, copytree, rmtree if sys.version_info < (2, 7): diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 3176502b9e7c..177bce2f005f 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -33,8 +33,11 @@ import org.apache.spark.util.ParentClassLoader /** * A ClassLoader that reads classes from a Hadoop FileSystem or Spark RPC endpoint, used to load * classes defined by the interpreter when the REPL is used. Allows the user to specify if user - * class path should be first. This class loader delegates getting/finding resources to parent - * loader, which makes sense until REPL never provide resource dynamically. + * class path should be first. + * This class loader delegates getting/finding resources to parent loader, which makes sense because + * the REPL never produce resources dynamically. One exception is when getting a Class file as + * resource stream, in which case we will try to fetch the Class file in the same way as loading + * the class, so that dynamically generated Classes from the REPL can be picked up. * * Note: [[ClassLoader]] will preferentially load class from parent. Only when parent is null or * the load failed, that it will call the overridden `findClass` function. To avoid the potential @@ -71,6 +74,30 @@ class ExecutorClassLoader( parentLoader.getResources(name) } + override def getResourceAsStream(name: String): InputStream = { + if (userClassPathFirst) { + val res = getClassResourceAsStreamLocally(name) + if (res != null) res else parentLoader.getResourceAsStream(name) + } else { + val res = parentLoader.getResourceAsStream(name) + if (res != null) res else getClassResourceAsStreamLocally(name) + } + } + + private def getClassResourceAsStreamLocally(name: String): InputStream = { + // Class files can be dynamically generated from the REPL. Allow this class loader to + // load such files for purposes other than loading the class. + try { + if (name.endsWith(".class")) fetchFn(name) else null + } catch { + // The helper functions referenced by fetchFn throw CNFE to indicate failure to fetch + // the class. It matches what IOException was supposed to be used for, and + // ClassLoader.getResourceAsStream() catches IOException and returns null in that case. + // So we follow that model and handle CNFE here. + case _: ClassNotFoundException => null + } + } + override def findClass(name: String): Class[_] = { if (userClassPathFirst) { findClassLocally(name).getOrElse(parentLoader.loadClass(name)) diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index e9ed01ff2233..4752495e8e75 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -208,6 +208,17 @@ class ExecutorClassLoaderSuite intercept[java.lang.ClassNotFoundException] { classLoader.loadClass("ReplFakeClassDoesNotExist").getConstructor().newInstance() } + + // classLoader.getResourceAsStream() should also be able to fetch the Class file + val fakeClassInputStream = classLoader.getResourceAsStream("ReplFakeClass2.class") + try { + val magic = new Array[Byte](4) + fakeClassInputStream.read(magic) + // first 4 bytes should match the magic number of Class file + assert(magic === Array[Byte](0xCA.toByte, 0xFE.toByte, 0xBA.toByte, 0xBE.toByte)) + } finally { + if (fakeClassInputStream != null) fakeClassInputStream.close() + } } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 4f3df729177f..a46cb6b3f401 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -260,4 +260,36 @@ class ReplSuite extends SparkFunSuite { assertContains("!!2!!", output2) } + test("SPARK-26633: ExecutorClassLoader.getResourceAsStream find REPL classes") { + val output = runInterpreterInPasteMode("local-cluster[1,1,1024]", + """ + |case class TestClass(value: Int) + | + |sc.parallelize(1 to 1).map { _ => + | val clz = classOf[TestClass] + | val name = clz.getName.replace('.', '/') + ".class"; + | val stream = clz.getClassLoader.getResourceAsStream(name) + | if (stream == null) { + | "failed: stream is null" + | } else { + | val magic = new Array[Byte](4) + | try { + | stream.read(magic) + | // the magic number of a Java Class file + | val expected = Array[Byte](0xCA.toByte, 0xFE.toByte, 0xBA.toByte, 0xBE.toByte) + | if (magic sameElements expected) { + | "successful" + | } else { + | "failed: unexpected contents from stream" + | } + | } finally { + | stream.close() + | } + | } + |}.collect() + """.stripMargin) + assertDoesNotContain("failed", output) + assertContains("successful", output) + } + } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index d52988df58d6..e664b647bd1e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -27,6 +27,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils @@ -153,12 +154,11 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) KUBERNETES_DRIVER_SUBMIT_CHECK.key -> "true", MEMORY_OVERHEAD_FACTOR.key -> overheadFactor.toString) - Seq("spark.jars", "spark.files").foreach { key => - conf.getOption(key).foreach { value => - val resolved = KubernetesUtils.resolveFileUrisAndPath(Utils.stringToSeq(value)) - if (resolved.nonEmpty) { - additionalProps.put(key, resolved.mkString(",")) - } + Seq(JARS, FILES).foreach { key => + val value = conf.get(key) + val resolved = KubernetesUtils.resolveFileUrisAndPath(value) + if (resolved.nonEmpty) { + additionalProps.put(key.key, resolved.mkString(",")) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala index 76b4ec98d494..bd3f8a1681e5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala @@ -109,21 +109,22 @@ private[spark] class DriverCommandFeatureStep(conf: KubernetesDriverConf) } private def additionalJavaProperties(resource: String): Map[String, String] = { - resourceType(APP_RESOURCE_TYPE_JAVA) ++ mergeFileList("spark.jars", Seq(resource)) + resourceType(APP_RESOURCE_TYPE_JAVA) ++ mergeFileList(JARS, Seq(resource)) } private def additionalPythonProperties(resource: String): Map[String, String] = { resourceType(APP_RESOURCE_TYPE_PYTHON) ++ - mergeFileList("spark.files", Seq(resource) ++ conf.pyFiles) + mergeFileList(FILES, Seq(resource) ++ conf.pyFiles) } private def additionalRProperties(resource: String): Map[String, String] = { - resourceType(APP_RESOURCE_TYPE_R) ++ mergeFileList("spark.files", Seq(resource)) + resourceType(APP_RESOURCE_TYPE_R) ++ mergeFileList(FILES, Seq(resource)) } - private def mergeFileList(key: String, filesToAdd: Seq[String]): Map[String, String] = { - val existing = Utils.stringToSeq(conf.get(key, "")) - Map(key -> (existing ++ filesToAdd).distinct.mkString(",")) + private def mergeFileList(key: ConfigEntry[Seq[String]], filesToAdd: Seq[String]) + : Map[String, String] = { + val existing = conf.get(key) + Map(key.key -> (existing ++ filesToAdd).distinct.mkString(",")) } private def resourceType(resType: String): Map[String, String] = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 27d59dd7f3e5..ccf88cc53f8c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils @@ -69,7 +70,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val expectedPortNames = Set( containerPort(DRIVER_PORT_NAME, DEFAULT_DRIVER_PORT), containerPort(BLOCK_MANAGER_PORT_NAME, DEFAULT_BLOCKMANAGER_PORT), - containerPort(UI_PORT_NAME, SparkUI.DEFAULT_PORT) + containerPort(UI_PORT_NAME, UI_PORT.defaultValue.get) ) val foundPortNames = configuredPod.container.getPorts.asScala.toSet assert(expectedPortNames === foundPortNames) @@ -142,7 +143,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val sparkConf = new SparkConf() .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod") .setJars(allJars) - .set("spark.files", allFiles.mkString(",")) + .set(FILES, allFiles) .set(CONTAINER_IMAGE, "spark-driver:latest") val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) @@ -153,8 +154,8 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { "spark.app.id" -> KubernetesTestConf.APP_ID, KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> kubernetesConf.resourceNamePrefix, "spark.kubernetes.submitInDriver" -> "true", - "spark.jars" -> "/opt/spark/jar1.jar,hdfs:///opt/spark/jar2.jar", - "spark.files" -> "https://localhost:9000/file1.txt,/opt/spark/file2.txt", + JARS.key -> "/opt/spark/jar1.jar,hdfs:///opt/spark/jar2.jar", + FILES.key -> "https://localhost:9000/file1.txt,/opt/spark/file2.txt", MEMORY_OVERHEAD_FACTOR.key -> MEMORY_OVERHEAD_FACTOR.defaultValue.get.toString) assert(additionalProperties === expectedSparkConf) } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index 250eaab9e980..e539c8e78dab 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -27,7 +27,9 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.JARS import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI.UI_ENABLED private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { @@ -67,8 +69,8 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl .set("spark.executor.cores", "1") .set("spark.executors.instances", "1") .set("spark.app.name", "spark-test-app") - .set("spark.ui.enabled", "true") .set(IS_TESTING.key, "false") + .set(UI_ENABLED.key, "true") .set("spark.kubernetes.submission.waitAppCompletion", "false") .set("spark.kubernetes.authenticate.driver.serviceAccountName", serviceAccountName) } @@ -85,7 +87,7 @@ private[spark] class SparkAppConf { def get(key: String): String = map.getOrElse(key, "") - def setJars(jars: Seq[String]): Unit = set("spark.jars", jars.mkString(",")) + def setJars(jars: Seq[String]): Unit = set(JARS.key, jars.mkString(",")) override def toString: String = map.toString diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 32ac4f37c5f9..bc1247ad7893 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -25,6 +25,7 @@ import org.apache.spark.deploy.mesos.config._ import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.mesos.MesosRestServer import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.scheduler.cluster.mesos._ import org.apache.spark.util.{CommandLineUtils, ShutdownHookManager, SparkUncaughtExceptionHandler, Utils} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala index dd0b2bad1ecb..2b8655ceee0c 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala @@ -63,11 +63,6 @@ package object config { .timeConf(TimeUnit.SECONDS) .createWithDefaultString("30s") - private[spark] val RECOVERY_MODE = - ConfigBuilder("spark.deploy.recoveryMode") - .stringConf - .createWithDefault("NONE") - private[spark] val DISPATCHER_WEBUI_URL = ConfigBuilder("spark.mesos.dispatcher.webui.url") .doc("Set the Spark Mesos dispatcher webui_url for interacting with the " + @@ -75,13 +70,6 @@ package object config { .stringConf .createOptional - private[spark] val ZOOKEEPER_URL = - ConfigBuilder("spark.deploy.zookeeper.url") - .doc("When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this " + - "configuration is used to set the zookeeper URL to connect to.") - .stringConf - .createOptional - private[spark] val HISTORY_SERVER_URL = ConfigBuilder("spark.mesos.dispatcher.historyServer.url") .doc("Set the URL of the history server. The dispatcher will then " + diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index a4aba3e9c0d0..3ff68348be7b 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -68,6 +68,10 @@ private[mesos] class MesosSubmitRequestServlet( private def newDriverId(submitDate: Date): String = f"driver-${createDateFormat.format(submitDate)}-${nextDriverNumber.incrementAndGet()}%04d" + // These defaults copied from YARN + private val MEMORY_OVERHEAD_FACTOR = 0.10 + private val MEMORY_OVERHEAD_MIN = 384 + /** * Build a driver description from the fields specified in the submit request. * @@ -98,6 +102,7 @@ private[mesos] class MesosSubmitRequestServlet( val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) val superviseDriver = sparkProperties.get(config.DRIVER_SUPERVISE.key) val driverMemory = sparkProperties.get(config.DRIVER_MEMORY.key) + val driverMemoryOverhead = sparkProperties.get(config.DRIVER_MEMORY_OVERHEAD.key) val driverCores = sparkProperties.get(config.DRIVER_CORES.key) val name = request.sparkProperties.getOrElse("spark.app.name", mainClass) @@ -112,13 +117,15 @@ private[mesos] class MesosSubmitRequestServlet( mainClass, appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts) val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) + val actualDriverMemoryOverhead = driverMemoryOverhead.map(_.toInt).getOrElse( + math.max((MEMORY_OVERHEAD_FACTOR * actualDriverMemory).toInt, MEMORY_OVERHEAD_MIN)) val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES) val submitDate = new Date() val submissionId = newDriverId(submitDate) new MesosDriverDescription( - name, appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, - command, request.sparkProperties, submissionId, submitDate) + name, appResource, actualDriverMemory + actualDriverMemoryOverhead, actualDriverCores, + actualSuperviseDriver, command, request.sparkProperties, submissionId, submitDate) } protected override def handleSubmit( diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala index 61ab3e87c571..123412f21e2a 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala @@ -26,6 +26,7 @@ import org.apache.zookeeper.KeeperException.NoNodeException import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.util.Utils /** @@ -94,13 +95,13 @@ private[spark] class ZookeeperMesosClusterPersistenceEngine( zk: CuratorFramework, conf: SparkConf) extends MesosClusterPersistenceEngine with Logging { - private val WORKING_DIR = - conf.get("spark.deploy.zookeeper.dir", "/spark_mesos_dispatcher") + "/" + baseDir + private val workingDir = + conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark_mesos_dispatcher") + "/" + baseDir - SparkCuratorUtil.mkdir(zk, WORKING_DIR) + SparkCuratorUtil.mkdir(zk, workingDir) def path(name: String): String = { - WORKING_DIR + "/" + name + workingDir + "/" + name } override def expunge(name: String): Unit = { @@ -129,6 +130,6 @@ private[spark] class ZookeeperMesosClusterPersistenceEngine( } override def fetchAll[T](): Iterable[T] = { - zk.getChildren.forPath(WORKING_DIR).asScala.flatMap(fetch[T]) + zk.getChildren.forPath(workingDir).asScala.flatMap(fetch[T]) } } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 021b1ac84805..8c961a541aa1 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -32,7 +32,7 @@ import org.apache.mesos.Protos.TaskStatus.Reason import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState} import org.apache.spark.deploy.mesos.{config, MesosDriverDescription} import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} -import org.apache.spark.internal.config.{CORES_MAX, EXECUTOR_LIBRARY_PATH, EXECUTOR_MEMORY} +import org.apache.spark.internal.config._ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.Utils @@ -432,7 +432,7 @@ private[spark] class MesosClusterScheduler( private def getDriverUris(desc: MesosDriverDescription): List[CommandInfo.URI] = { val confUris = List(conf.getOption("spark.mesos.uris"), desc.conf.getOption("spark.mesos.uris"), - desc.conf.getOption("spark.submit.pyFiles")).flatMap( + Some(desc.conf.get(SUBMIT_PYTHON_FILES).mkString(","))).flatMap( _.map(_.split(",").map(_.trim)) ).flatten @@ -534,16 +534,16 @@ private[spark] class MesosClusterScheduler( desc.conf.getOption(CORES_MAX.key).foreach { v => options ++= Seq("--total-executor-cores", v) } - desc.conf.getOption("spark.submit.pyFiles").foreach { pyFiles => - val formattedFiles = pyFiles.split(",") - .map { path => new File(sandboxPath, path.split("/").last).toString() } - .mkString(",") - options ++= Seq("--py-files", formattedFiles) - } + + val pyFiles = desc.conf.get(SUBMIT_PYTHON_FILES) + val formattedFiles = pyFiles.map { path => + new File(sandboxPath, path.split("/").last).toString() + }.mkString(",") + options ++= Seq("--py-files", formattedFiles) // --conf val replicatedOptionsBlacklist = Set( - "spark.jars", // Avoids duplicate classes in classpath + JARS.key, // 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 ) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.scala new file mode 100644 index 000000000000..1f83149a0565 --- /dev/null +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.rest.mesos + +import javax.servlet.http.HttpServletResponse + +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.TestPrematureExit +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.deploy.rest.{CreateSubmissionRequest, CreateSubmissionResponse, SubmitRestProtocolMessage, SubmitRestProtocolResponse} +import org.apache.spark.internal.config +import org.apache.spark.scheduler.cluster.mesos.{MesosClusterPersistenceEngineFactory, MesosClusterScheduler} + +class MesosRestServerSuite extends SparkFunSuite + with TestPrematureExit with MockitoSugar { + + test("test default driver overhead memory") { + testOverheadMemory(new SparkConf(), "2000M", 2384) + } + + test("test driver overhead memory with overhead factor") { + testOverheadMemory(new SparkConf(), "5000M", 5500) + } + + test("test configured driver overhead memory") { + val conf = new SparkConf() + conf.set(config.DRIVER_MEMORY_OVERHEAD.key, "1000") + testOverheadMemory(conf, "2000M", 3000) + } + + def testOverheadMemory(conf: SparkConf, driverMemory: String, expectedResult: Int) { + conf.set("spark.master", "testmaster") + conf.set("spark.app.name", "testapp") + conf.set(config.DRIVER_MEMORY.key, driverMemory) + var actualMem = 0 + class TestMesosClusterScheduler extends MesosClusterScheduler( + mock[MesosClusterPersistenceEngineFactory], conf) { + override def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = { + actualMem = desc.mem + mock[CreateSubmissionResponse] + } + } + + class TestServlet extends MesosSubmitRequestServlet(new TestMesosClusterScheduler, conf) { + override def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + super.handleSubmit(requestMessageJson, requestMessage, responseServlet) + } + + override def findUnknownFields( + requestJson: String, + requestMessage: SubmitRestProtocolMessage): Array[String] = { + Array() + } + } + val servlet = new TestServlet() + val request = new CreateSubmissionRequest() + request.appResource = "testresource" + request.mainClass = "mainClass" + request.appArgs = Array("appArgs") + request.environmentVariables = Map("envVar" -> "envVal") + request.sparkProperties = conf.getAll.toMap + servlet.handleSubmit("null", request, null) + assert(actualMem == expectedResult) + } +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 1ece7bdc979c..01b91887985f 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -44,6 +44,7 @@ import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend} @@ -224,7 +225,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends val attemptID = if (isClusterMode) { // Set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box - System.setProperty("spark.ui.port", "0") + System.setProperty(UI_PORT.key, "0") // Set the master and deploy mode property to match the requested mode. System.setProperty("spark.master", "yarn") @@ -620,7 +621,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends d.send(AddWebUIFilter(amFilter, params.toMap, proxyBase)) case None => - System.setProperty("spark.ui.filters", amFilter) + System.setProperty(UI_FILTERS.key, amFilter) params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) } } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 84921800a471..79922920267e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -68,7 +68,7 @@ private[spark] class Client( private val yarnClient = YarnClient.createYarnClient private val hadoopConf = new YarnConfiguration(SparkHadoopUtil.newConfiguration(sparkConf)) - private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster" + private val isClusterMode = sparkConf.get(SUBMIT_DEPLOY_MODE) == "cluster" // AM related configurations private val amMemory = if (isClusterMode) { @@ -1532,8 +1532,8 @@ private[spark] class YarnClusterApplication extends SparkApplication { override def start(args: Array[String], conf: SparkConf): Unit = { // SparkSubmit would use yarn cache to distribute files & jars in yarn mode, // so remove them from sparkConf here for yarn mode. - conf.remove("spark.jars") - conf.remove("spark.files") + conf.remove(JARS) + conf.remove(FILES) new Client(new ClientArguments(args), conf).run() } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index a9ff3023a581..a7bed75a02ad 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -33,6 +33,7 @@ import org.apache.spark.SparkContext import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -175,7 +176,7 @@ private[spark] abstract class YarnSchedulerBackend( filterParams != null && filterParams.nonEmpty if (hasFilter) { // SPARK-26255: Append user provided filters(spark.ui.filters) with yarn filter. - val allFilters = filterName + "," + conf.get("spark.ui.filters", "") + val allFilters = Seq(filterName) ++ conf.get(UI_FILTERS) logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") // For already installed handlers, prepend the filter. @@ -186,7 +187,7 @@ private[spark] abstract class YarnSchedulerBackend( filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } - conf.set("spark.ui.filters", allFilters) + conf.set(UI_FILTERS, allFilters) ui.getHandlers.map(_.getServletHandler()).foreach { h => val holder = new FilterHolder() diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 9acd99546c03..25827fd25e6b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -40,6 +40,7 @@ import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite, TestUtils} import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.config._ import org.apache.spark.util.{SparkConfWithEnv, Utils} class ClientSuite extends SparkFunSuite with Matchers { @@ -368,7 +369,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val resources = Map("fpga" -> 2, "gpu" -> 3) ResourceRequestTestHelper.initializeResourceTypes(resources.keys.toSeq) - val conf = new SparkConf().set("spark.submit.deployMode", deployMode) + val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, deployMode) resources.foreach { case (name, v) => conf.set(prefix + name, v.toString) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 506b27c677f5..faddb8fc7e22 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -36,6 +36,8 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher._ import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} @@ -192,7 +194,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { val propsFile = createConfFile() val handle = new SparkLauncher(env) .setSparkHome(sys.props("spark.test.home")) - .setConf("spark.ui.enabled", "false") + .setConf(UI_ENABLED.key, "false") .setPropertiesFile(propsFile) .setMaster("yarn") .setDeployMode("client") @@ -441,7 +443,7 @@ private object YarnClusterDriver extends Logging with Matchers { // If we are running in yarn-cluster mode, verify that driver logs links and present and are // in the expected format. - if (conf.get("spark.submit.deployMode") == "cluster") { + if (conf.get(SUBMIT_DEPLOY_MODE) == "cluster") { assert(listener.driverLogs.nonEmpty) val driverLogs = listener.driverLogs.get assert(driverLogs.size === 2) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 61c0c43f7c04..de7ff8238c9e 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -30,6 +30,8 @@ import org.scalatest.Matchers import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.util.{ResetSystemProperties, Utils} class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging @@ -83,7 +85,7 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging // spark acls on, just pick up default user val sparkConf = new SparkConf() - sparkConf.set("spark.acls.enable", "true") + sparkConf.set(ACLS_ENABLE, true) val securityMgr = new SecurityManager(sparkConf) val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) @@ -111,9 +113,9 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging // default spark acls are on and specify acls val sparkConf = new SparkConf() - sparkConf.set("spark.acls.enable", "true") - sparkConf.set("spark.ui.view.acls", "user1,user2") - sparkConf.set("spark.modify.acls", "user3,user4") + sparkConf.set(ACLS_ENABLE, true) + sparkConf.set(UI_VIEW_ACLS, Seq("user1", "user2")) + sparkConf.set(MODIFY_ACLS, Seq("user3", "user4")) val securityMgr = new SecurityManager(sparkConf) val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 5395e4035e68..d6edddfc1ae6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -40,7 +40,6 @@ public final class UnsafeExternalRowSorter { - static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; /** * If positive, forces records to be spilled to disk at the given frequency (measured in numbers * of records). This is only intended to be used in tests. @@ -112,8 +111,7 @@ private UnsafeExternalRowSorter( taskContext, recordComparatorSupplier, prefixComparator, - sparkEnv.conf().getInt("spark.shuffle.sort.initialBufferSize", - DEFAULT_INITIAL_SORT_BUFFER_SIZE), + (int) sparkEnv.conf().get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()), pageSizeBytes, (int) SparkEnv.get().conf().get( package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD()), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index ea8c369ee49e..7ae5924b20fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -86,4 +86,13 @@ object BindReferences extends Logging { } }.asInstanceOf[A] // Kind of a hack, but safe. TODO: Tighten return type when possible. } + + /** + * A helper function to bind given expressions to an input schema. + */ + def bindReferences[A <: Expression]( + expressions: Seq[A], + input: AttributeSeq): Seq[A] = { + expressions.map(BindReferences.bindReference(_, input)) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index ee463bf5eb6a..ff6a68b29020 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -230,12 +230,15 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // [[func]] assumes the input is no longer null because eval already does the null check. @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) + private lazy val dateFormatter = DateFormatter() + private lazy val timestampFormatter = TimestampFormatter(timeZone) + // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes) - case DateType => buildCast[Int](_, d => UTF8String.fromString(DateTimeUtils.dateToString(d))) + case DateType => buildCast[Int](_, d => UTF8String.fromString(dateFormatter.format(d))) case TimestampType => buildCast[Long](_, - t => UTF8String.fromString(DateTimeUtils.timestampToString(t, timeZone))) + t => UTF8String.fromString(DateTimeUtils.timestampToString(timestampFormatter, t))) case ArrayType(et, _) => buildCast[ArrayData](_, array => { val builder = new UTF8StringBuilder @@ -843,12 +846,16 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String case BinaryType => (c, evPrim, evNull) => code"$evPrim = UTF8String.fromBytes($c);" case DateType => - (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString( - org.apache.spark.sql.catalyst.util.DateTimeUtils.dateToString($c));""" + val df = JavaCode.global( + ctx.addReferenceObj("dateFormatter", dateFormatter), + dateFormatter.getClass) + (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString(${df}.format($c));""" case TimestampType => - val tz = JavaCode.global(ctx.addReferenceObj("timeZone", timeZone), timeZone.getClass) + val tf = JavaCode.global( + ctx.addReferenceObj("timestampFormatter", timestampFormatter), + timestampFormatter.getClass) (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString( - org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));""" + org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($tf, $c));""" case ArrayType(et, _) => (c, evPrim, evNull) => { val buffer = ctx.freshVariable("buffer", classOf[UTF8StringBuilder]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala index 122a564da61b..5c8aa4e2e9d8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp @@ -30,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp */ class InterpretedMutableProjection(expressions: Seq[Expression]) extends MutableProjection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = - this(toBoundExprs(expressions, inputSchema)) + this(bindReferences(expressions, inputSchema)) private[this] val buffer = new Array[Any](expressions.size) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index b48f7ba655b2..eaaf94baac21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateMutableProjection, GenerateSafeProjection, GenerateUnsafeProjection} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} @@ -30,7 +31,7 @@ import org.apache.spark.sql.types.{DataType, StructType} */ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = - this(expressions.map(BindReferences.bindReference(_, inputSchema))) + this(bindReferences(expressions, inputSchema)) override def initialize(partitionIndex: Int): Unit = { expressions.foreach(_.foreach { @@ -99,7 +100,7 @@ object MutableProjection * `inputSchema`. */ def create(exprs: Seq[Expression], inputSchema: Seq[Attribute]): MutableProjection = { - create(toBoundExprs(exprs, inputSchema)) + create(bindReferences(exprs, inputSchema)) } } @@ -162,7 +163,7 @@ object UnsafeProjection * `inputSchema`. */ def create(exprs: Seq[Expression], inputSchema: Seq[Attribute]): UnsafeProjection = { - create(toBoundExprs(exprs, inputSchema)) + create(bindReferences(exprs, inputSchema)) } } @@ -203,6 +204,6 @@ object SafeProjection extends CodeGeneratorWithInterpretedFallback[Seq[Expressio * `inputSchema`. */ def create(exprs: Seq[Expression], inputSchema: Seq[Attribute]): Projection = { - create(toBoundExprs(exprs, inputSchema)) + create(bindReferences(exprs, inputSchema)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index d588e7f08130..838bd1c679e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp // MutableProjection is not accessible in Java @@ -35,7 +36,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = - in.map(BindReferences.bindReference(_, inputSchema)) + bindReferences(in, inputSchema) def generate( expressions: Seq[Expression], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 283fd2a6e938..b66b80ad31dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -25,6 +25,7 @@ import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -46,7 +47,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR in.map(ExpressionCanonicalizer.execute(_).asInstanceOf[SortOrder]) protected def bind(in: Seq[SortOrder], inputSchema: Seq[Attribute]): Seq[SortOrder] = - in.map(BindReferences.bindReference(_, inputSchema)) + bindReferences(in, inputSchema) /** * Creates a code gen ordering for sorting this schema, in ascending order. @@ -188,7 +189,7 @@ class LazilyGeneratedOrdering(val ordering: Seq[SortOrder]) extends Ordering[InternalRow] with KryoSerializable { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = - this(ordering.map(BindReferences.bindReference(_, inputSchema))) + this(bindReferences(ordering, inputSchema)) @transient private[this] var generatedOrdering = GenerateOrdering.generate(ordering) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index 39778661d1c4..e285398ba195 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -21,6 +21,7 @@ import scala.annotation.tailrec import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} @@ -41,7 +42,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = - in.map(BindReferences.bindReference(_, inputSchema)) + bindReferences(in, inputSchema) private def createCodeForStruct( ctx: CodegenContext, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 0ecd0de8d820..fb1d8a3c8e73 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.types._ @@ -317,7 +318,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = - in.map(BindReferences.bindReference(_, inputSchema)) + bindReferences(in, inputSchema) def generate( expressions: Seq[Expression], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 8fc0112c0257..e7583628a934 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -562,7 +562,7 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti copy(timeZoneId = Option(timeZoneId)) override protected def nullSafeEval(timestamp: Any, format: Any): Any = { - val df = TimestampFormatter(format.toString, timeZone, Locale.US) + val df = TimestampFormatter(format.toString, timeZone) UTF8String.fromString(df.format(timestamp.asInstanceOf[Long])) } @@ -614,9 +614,7 @@ case class ToUnixTimestamp( /** * Converts time string with given pattern to Unix time stamp (in seconds), returns null if fail. - * See [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html] - * if SQL config spark.sql.legacy.timeParser.enabled is set to true otherwise - * [https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html]. + * See [https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html]. * Note that hive Language Manual says it returns 0 if fail, but in fact it returns null. * If the second parameter is missing, use "yyyy-MM-dd HH:mm:ss". * If no parameters provided, the first parameter will be current_timestamp. @@ -667,7 +665,7 @@ abstract class UnixTime private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] private lazy val formatter: TimestampFormatter = try { - TimestampFormatter(constFormat.toString, timeZone, Locale.US) + TimestampFormatter(constFormat.toString, timeZone) } catch { case NonFatal(_) => null } @@ -700,7 +698,7 @@ abstract class UnixTime } else { val formatString = f.asInstanceOf[UTF8String].toString try { - TimestampFormatter(formatString, timeZone, Locale.US).parse( + TimestampFormatter(formatString, timeZone).parse( t.asInstanceOf[UTF8String].toString) / MICROS_PER_SECOND } catch { case NonFatal(_) => null @@ -821,7 +819,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] private lazy val formatter: TimestampFormatter = try { - TimestampFormatter(constFormat.toString, timeZone, Locale.US) + TimestampFormatter(constFormat.toString, timeZone) } catch { case NonFatal(_) => null } @@ -847,7 +845,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ null } else { try { - UTF8String.fromString(TimestampFormatter(f.toString, timeZone, Locale.US) + UTF8String.fromString(TimestampFormatter(f.toString, timeZone) .format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) } catch { case NonFatal(_) => null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index e0cab537ce1c..3403349c8974 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -548,23 +548,15 @@ case class JsonToStructs( s"Input schema ${nullableSchema.catalogString} must be a struct, an array or a map.") } - @transient - private lazy val castRow = nullableSchema match { - case _: StructType => (row: InternalRow) => row - case _: ArrayType => (row: InternalRow) => row.getArray(0) - case _: MapType => (row: InternalRow) => row.getMap(0) - } - // This converts parsed rows to the desired output by the given schema. - private def convertRow(rows: Iterator[InternalRow]) = { - if (rows.hasNext) { - val result = rows.next() - // JSON's parser produces one record only. - assert(!rows.hasNext) - castRow(result) - } else { - throw new IllegalArgumentException("Expected one row from JSON parser.") - } + @transient + lazy val converter = nullableSchema match { + case _: StructType => + (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next() else null + case _: ArrayType => + (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getArray(0) else null + case _: MapType => + (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getMap(0) else null } val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) @@ -600,7 +592,7 @@ case class JsonToStructs( copy(timeZoneId = Option(timeZoneId)) override def nullSafeEval(json: Any): Any = { - convertRow(parser.parse(json.asInstanceOf[UTF8String])) + converter(parser.parse(json.asInstanceOf[UTF8String])) } override def inputTypes: Seq[AbstractDataType] = StringType :: Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala index e24a3de3cfdb..c8d667143f45 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.types._ @@ -27,7 +28,7 @@ import org.apache.spark.sql.types._ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow] { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = - this(ordering.map(BindReferences.bindReference(_, inputSchema))) + this(bindReferences(ordering, inputSchema)) def compare(a: InternalRow, b: InternalRow): Int = { var i = 0 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index bf18e8bcb52d..932c36473724 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -85,13 +85,6 @@ package object expressions { override def apply(row: InternalRow): InternalRow = row } - /** - * A helper function to bind given expressions to an input schema. - */ - def toBoundExprs(exprs: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = { - exprs.map(BindReferences.bindReference(_, inputSchema)) - } - /** * Helper functions for working with `Seq[Attribute]`. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 3f245e1400fa..8cf758e26e29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -399,7 +399,7 @@ class JacksonParser( // a null first token is equivalent to testing for input.trim.isEmpty // but it works on any token stream and not just strings parser.nextToken() match { - case null => throw new RuntimeException("Not found any JSON token") + case null => Nil case _ => rootConverter.apply(parser) match { case null => throw new RuntimeException("Root converter returned null") case rows => rows diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index 743d3ce944fe..6540e95b01e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -31,40 +31,6 @@ import org.apache.spark.sql.internal.SQLConf * Cost-based join reorder. * We may have several join reorder algorithms in the future. This class is the entry of these * algorithms, and chooses which one to use. - * - * Note that join strategy hints, e.g. the broadcast hint, do not interfere with the reordering. - * Such hints will be applied on the equivalent counterparts (i.e., join between the same relations - * regardless of the join order) of the original nodes after reordering. - * For example, the plan before reordering is like: - * - * Join - * / \ - * Hint1 t4 - * / - * Join - * / \ - * Join t3 - * / \ - * Hint2 t2 - * / - * t1 - * - * The original join order as illustrated above is "((t1 JOIN t2) JOIN t3) JOIN t4", and after - * reordering, the new join order is "((t1 JOIN t3) JOIN t2) JOIN t4", so the new plan will be like: - * - * Join - * / \ - * Hint1 t4 - * / - * Join - * / \ - * Join t2 - * / \ - * t1 t3 - * - * "Hint1" is applied on "(t1 JOIN t3) JOIN t2" as it is equivalent to the original hinted node, - * "(t1 JOIN t2) JOIN t3"; while "Hint2" has disappeared from the new plan since there is no - * equivalent node to "t1 JOIN t2". */ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { @@ -74,30 +40,24 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { if (!conf.cboEnabled || !conf.joinReorderEnabled) { plan } else { - // Use a map to track the hints on the join items. - val hintMap = new mutable.HashMap[AttributeSet, HintInfo] val result = plan transformDown { // Start reordering with a joinable item, which is an InnerLike join with conditions. - case j @ Join(_, _, _: InnerLike, Some(cond), _) => - reorder(j, j.output, hintMap) - case p @ Project(projectList, Join(_, _, _: InnerLike, Some(cond), _)) - if projectList.forall(_.isInstanceOf[Attribute]) => - reorder(p, p.output, hintMap) + // Avoid reordering if a join hint is present. + case j @ Join(_, _, _: InnerLike, Some(cond), hint) if hint == JoinHint.NONE => + reorder(j, j.output) + case p @ Project(projectList, Join(_, _, _: InnerLike, Some(cond), hint)) + if projectList.forall(_.isInstanceOf[Attribute]) && hint == JoinHint.NONE => + reorder(p, p.output) } // After reordering is finished, convert OrderedJoin back to Join. result transform { - case OrderedJoin(left, right, jt, cond) => - val joinHint = JoinHint(hintMap.get(left.outputSet), hintMap.get(right.outputSet)) - Join(left, right, jt, cond, joinHint) + case OrderedJoin(left, right, jt, cond) => Join(left, right, jt, cond, JoinHint.NONE) } } } - private def reorder( - plan: LogicalPlan, - output: Seq[Attribute], - hintMap: mutable.HashMap[AttributeSet, HintInfo]): LogicalPlan = { - val (items, conditions) = extractInnerJoins(plan, hintMap) + private def reorder(plan: LogicalPlan, output: Seq[Attribute]): LogicalPlan = { + val (items, conditions) = extractInnerJoins(plan) val result = // Do reordering if the number of items is appropriate and join conditions exist. // We also need to check if costs of all items can be evaluated. @@ -115,20 +75,16 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { * Extracts items of consecutive inner joins and join conditions. * This method works for bushy trees and left/right deep trees. */ - private def extractInnerJoins( - plan: LogicalPlan, - hintMap: mutable.HashMap[AttributeSet, HintInfo]): (Seq[LogicalPlan], Set[Expression]) = { + private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = { plan match { - case Join(left, right, _: InnerLike, Some(cond), hint) => - hint.leftHint.foreach(hintMap.put(left.outputSet, _)) - hint.rightHint.foreach(hintMap.put(right.outputSet, _)) - val (leftPlans, leftConditions) = extractInnerJoins(left, hintMap) - val (rightPlans, rightConditions) = extractInnerJoins(right, hintMap) + case Join(left, right, _: InnerLike, Some(cond), _) => + val (leftPlans, leftConditions) = extractInnerJoins(left) + val (rightPlans, rightConditions) = extractInnerJoins(right) (leftPlans ++ rightPlans, splitConjunctivePredicates(cond).toSet ++ leftConditions ++ rightConditions) case Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond), _)) if projectList.forall(_.isInstanceOf[Attribute]) => - extractInnerJoins(j, hintMap) + extractInnerJoins(j) case _ => (Seq(plan), Set()) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index d92f7f860b1b..20f1221be425 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -561,6 +561,11 @@ object ColumnPruning extends Rule[LogicalPlan] { case d @ DeserializeToObject(_, _, child) if !child.outputSet.subsetOf(d.references) => d.copy(child = prunedChild(child, d.references)) + case p @ Project(_, s: SerializeFromObject) if p.references != s.outputSet => + val usedRefs = p.references + val prunedSerializer = s.serializer.filter(usedRefs.contains) + p.copy(child = SerializeFromObject(prunedSerializer, s.child)) + // Prunes the unused columns from child of Aggregate/Expand/Generate/ScriptTransformation case a @ Aggregate(_, _, child) if !child.outputSet.subsetOf(a.references) => a.copy(child = prunedChild(child, a.references)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 82aefca8a1af..251ece315f6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -43,13 +43,11 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { * * @param input a list of LogicalPlans to inner join and the type of inner join. * @param conditions a list of condition for join. - * @param hintMap a map of relation output attribute sets to their corresponding hints. */ @tailrec final def createOrderedJoin( input: Seq[(LogicalPlan, InnerLike)], - conditions: Seq[Expression], - hintMap: Map[AttributeSet, HintInfo]): LogicalPlan = { + conditions: Seq[Expression]): LogicalPlan = { assert(input.size >= 2) if (input.size == 2) { val (joinConditions, others) = conditions.partition(canEvaluateWithinJoin) @@ -58,8 +56,8 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { case (Inner, Inner) => Inner case (_, _) => Cross } - val join = Join(left, right, innerJoinType, joinConditions.reduceLeftOption(And), - JoinHint(hintMap.get(left.outputSet), hintMap.get(right.outputSet))) + val join = Join(left, right, innerJoinType, + joinConditions.reduceLeftOption(And), JoinHint.NONE) if (others.nonEmpty) { Filter(others.reduceLeft(And), join) } else { @@ -82,27 +80,27 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val joinedRefs = left.outputSet ++ right.outputSet val (joinConditions, others) = conditions.partition( e => e.references.subsetOf(joinedRefs) && canEvaluateWithinJoin(e)) - val joined = Join(left, right, innerJoinType, joinConditions.reduceLeftOption(And), - JoinHint(hintMap.get(left.outputSet), hintMap.get(right.outputSet))) + val joined = Join(left, right, innerJoinType, + joinConditions.reduceLeftOption(And), JoinHint.NONE) // should not have reference to same logical plan - createOrderedJoin(Seq((joined, Inner)) ++ rest.filterNot(_._1 eq right), others, hintMap) + createOrderedJoin(Seq((joined, Inner)) ++ rest.filterNot(_._1 eq right), others) } } def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p @ ExtractFiltersAndInnerJoins(input, conditions, hintMap) + case p @ ExtractFiltersAndInnerJoins(input, conditions) if input.size > 2 && conditions.nonEmpty => val reordered = if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) { val starJoinPlan = StarSchemaDetection.reorderStarJoins(input, conditions) if (starJoinPlan.nonEmpty) { val rest = input.filterNot(starJoinPlan.contains(_)) - createOrderedJoin(starJoinPlan ++ rest, conditions, hintMap) + createOrderedJoin(starJoinPlan ++ rest, conditions) } else { - createOrderedJoin(input, conditions, hintMap) + createOrderedJoin(input, conditions) } } else { - createOrderedJoin(input, conditions, hintMap) + createOrderedJoin(input, conditions) } if (p.sameOutput(reordered)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index a27c6d3c3671..24bbe116ad89 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.parser -import java.sql.{Date, Timestamp} import java.util.Locale import javax.xml.bind.DatatypeConverter @@ -37,9 +36,10 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getTimeZone, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.random.RandomSampler /** @@ -1552,12 +1552,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) { val value = string(ctx.STRING) val valueType = ctx.identifier.getText.toUpperCase(Locale.ROOT) + def toLiteral[T](f: UTF8String => Option[T], t: DataType): Literal = { + f(UTF8String.fromString(value)).map(Literal(_, t)).getOrElse { + throw new ParseException(s"Cannot parse the $valueType value: $value", ctx) + } + } try { valueType match { - case "DATE" => - Literal(Date.valueOf(value)) + case "DATE" => toLiteral(stringToDate, DateType) case "TIMESTAMP" => - Literal(Timestamp.valueOf(value)) + val timeZone = getTimeZone(SQLConf.get.sessionLocalTimeZone) + toLiteral(stringToTimestamp(_, timeZone), TimestampType) case "X" => val padding = if (value.length % 2 != 0) "0" else "" Literal(DatatypeConverter.parseHexBinary(padding + value)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 95be0a52cb2e..a816922f49ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -166,35 +166,27 @@ object ExtractFiltersAndInnerJoins extends PredicateHelper { * was involved in an explicit cross join. Also returns the entire list of join conditions for * the left-deep tree. */ - def flattenJoin( - plan: LogicalPlan, - hintMap: mutable.HashMap[AttributeSet, HintInfo], - parentJoinType: InnerLike = Inner) + def flattenJoin(plan: LogicalPlan, parentJoinType: InnerLike = Inner) : (Seq[(LogicalPlan, InnerLike)], Seq[Expression]) = plan match { - case Join(left, right, joinType: InnerLike, cond, hint) => - val (plans, conditions) = flattenJoin(left, hintMap, joinType) - hint.leftHint.map(hintMap.put(left.outputSet, _)) - hint.rightHint.map(hintMap.put(right.outputSet, _)) + case Join(left, right, joinType: InnerLike, cond, hint) if hint == JoinHint.NONE => + val (plans, conditions) = flattenJoin(left, joinType) (plans ++ Seq((right, joinType)), conditions ++ cond.toSeq.flatMap(splitConjunctivePredicates)) - case Filter(filterCondition, j @ Join(_, _, _: InnerLike, _, _)) => - val (plans, conditions) = flattenJoin(j, hintMap) + case Filter(filterCondition, j @ Join(_, _, _: InnerLike, _, hint)) if hint == JoinHint.NONE => + val (plans, conditions) = flattenJoin(j) (plans, conditions ++ splitConjunctivePredicates(filterCondition)) case _ => (Seq((plan, parentJoinType)), Seq.empty) } def unapply(plan: LogicalPlan) - : Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression], Map[AttributeSet, HintInfo])] + : Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression])] = plan match { - case f @ Filter(filterCondition, j @ Join(_, _, joinType: InnerLike, _, _)) => - val hintMap = new mutable.HashMap[AttributeSet, HintInfo] - val flattened = flattenJoin(f, hintMap) - Some((flattened._1, flattened._2, hintMap.toMap)) - case j @ Join(_, _, joinType, _, _) => - val hintMap = new mutable.HashMap[AttributeSet, HintInfo] - val flattened = flattenJoin(j, hintMap) - Some((flattened._1, flattened._2, hintMap.toMap)) + case f @ Filter(filterCondition, j @ Join(_, _, joinType: InnerLike, _, hint)) + if hint == JoinHint.NONE => + Some(flattenJoin(f)) + case j @ Join(_, _, joinType, _, hint) if hint == JoinHint.NONE => + Some(flattenJoin(j)) case _ => None } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index db9255281488..9535a369cb2e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -20,11 +20,7 @@ package org.apache.spark.sql.catalyst.util import java.time.{Instant, ZoneId} import java.util.Locale -import scala.util.Try - -import org.apache.commons.lang3.time.FastDateFormat - -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToDays sealed trait DateFormatter extends Serializable { def parse(s: String): Int // returns days since epoch @@ -44,11 +40,7 @@ class Iso8601DateFormatter( toInstantWithZoneId(temporalAccessor, UTC) } - override def parse(s: String): Int = { - val seconds = toInstant(s).getEpochSecond - val days = Math.floorDiv(seconds, DateTimeUtils.SECONDS_PER_DAY) - days.toInt - } + override def parse(s: String): Int = instantToDays(toInstant(s)) override def format(days: Int): String = { val instant = Instant.ofEpochSecond(days * DateTimeUtils.SECONDS_PER_DAY) @@ -56,43 +48,15 @@ class Iso8601DateFormatter( } } -class LegacyDateFormatter(pattern: String, locale: Locale) extends DateFormatter { - @transient - private lazy val format = FastDateFormat.getInstance(pattern, locale) - - override def parse(s: String): Int = { - val milliseconds = format.parse(s).getTime - DateTimeUtils.millisToDays(milliseconds) - } +object DateFormatter { + val defaultPattern: String = "yyyy-MM-dd" + val defaultLocale: Locale = Locale.US - override def format(days: Int): String = { - val date = DateTimeUtils.toJavaDate(days) - format.format(date) + def apply(format: String, locale: Locale): DateFormatter = { + new Iso8601DateFormatter(format, locale) } -} -class LegacyFallbackDateFormatter( - pattern: String, - locale: Locale) extends LegacyDateFormatter(pattern, locale) { - override def parse(s: String): Int = { - Try(super.parse(s)).orElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - Try(DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(s).getTime)) - }.getOrElse { - // In Spark 1.5.0, we store the data as number of days since epoch in string. - // So, we just convert it to Int. - s.toInt - } - } -} + def apply(format: String): DateFormatter = apply(format, defaultLocale) -object DateFormatter { - def apply(format: String, locale: Locale): DateFormatter = { - if (SQLConf.get.legacyTimeParserEnabled) { - new LegacyFallbackDateFormatter(format, locale) - } else { - new Iso8601DateFormatter(format, locale) - } - } + def apply(): DateFormatter = apply(defaultPattern) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index e95117f95cdb..867647921349 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -18,13 +18,12 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} -import java.text.{DateFormat, SimpleDateFormat} +import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZonedDateTime} +import java.time.Year.isLeap +import java.time.temporal.IsoFields import java.util.{Calendar, Locale, TimeZone} -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.function.{Function => JFunction} -import javax.xml.bind.DatatypeConverter - -import scala.annotation.tailrec import org.apache.spark.unsafe.types.UTF8String @@ -53,30 +52,12 @@ object DateTimeUtils { final val NANOS_PER_MICROS = 1000L final val MILLIS_PER_DAY = SECONDS_PER_DAY * 1000L - // number of days in 400 years by Gregorian calendar - final val daysIn400Years: Int = 146097 - - // In the Julian calendar every year that is exactly divisible by 4 is a leap year without any - // exception. But in the Gregorian calendar every year that is exactly divisible by four - // is a leap year, except for years that are exactly divisible by 100, but these centurial years - // are leap years if they are exactly divisible by 400. - // So there are 3 extra days in the Julian calendar within a 400 years cycle compared to the - // Gregorian calendar. - final val extraLeapDaysIn400YearsJulian = 3 - - // number of days in 400 years by Julian calendar - final val daysIn400YearsInJulian: Int = daysIn400Years + extraLeapDaysIn400YearsJulian - // number of days between 1.1.1970 and 1.1.2001 final val to2001 = -11323 // this is year -17999, calculation: 50 * daysIn400Year final val YearZero = -17999 final val toYearZero = to2001 + 7304850 - - // days to year -17999 in Julian calendar - final val toYearZeroInJulian = toYearZero + 49 * extraLeapDaysIn400YearsJulian - final val TimeZoneGMT = TimeZone.getTimeZone("GMT") final val TimeZoneUTC = TimeZone.getTimeZone("UTC") final val MonthOf31Days = Set(1, 3, 5, 7, 8, 10, 12) @@ -85,39 +66,6 @@ object DateTimeUtils { def defaultTimeZone(): TimeZone = TimeZone.getDefault() - // Reuse the Calendar object in each thread as it is expensive to create in each method call. - private val threadLocalGmtCalendar = new ThreadLocal[Calendar] { - override protected def initialValue: Calendar = { - Calendar.getInstance(TimeZoneGMT) - } - } - - // `SimpleDateFormat` is not thread-safe. - private val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { - override def initialValue(): SimpleDateFormat = { - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) - } - } - - def getThreadLocalTimestampFormat(timeZone: TimeZone): DateFormat = { - val sdf = threadLocalTimestampFormat.get() - sdf.setTimeZone(timeZone) - sdf - } - - // `SimpleDateFormat` is not thread-safe. - private val threadLocalDateFormat = new ThreadLocal[DateFormat] { - override def initialValue(): SimpleDateFormat = { - new SimpleDateFormat("yyyy-MM-dd", Locale.US) - } - } - - def getThreadLocalDateFormat(timeZone: TimeZone): DateFormat = { - val sdf = threadLocalDateFormat.get() - sdf.setTimeZone(timeZone) - sdf - } - private val computedTimeZones = new ConcurrentHashMap[String, TimeZone] private val computeTimeZone = new JFunction[String, TimeZone] { override def apply(timeZoneId: String): TimeZone = TimeZone.getTimeZone(timeZoneId) @@ -149,24 +97,11 @@ object DateTimeUtils { millisLocal - getOffsetFromLocalMillis(millisLocal, timeZone) } - def dateToString(days: SQLDate): String = - getThreadLocalDateFormat(defaultTimeZone()).format(toJavaDate(days)) - - def dateToString(days: SQLDate, timeZone: TimeZone): String = { - getThreadLocalDateFormat(timeZone).format(toJavaDate(days)) - } - - // Converts Timestamp to string according to Hive TimestampWritable convention. - def timestampToString(us: SQLTimestamp): String = { - timestampToString(us, defaultTimeZone()) - } - // Converts Timestamp to string according to Hive TimestampWritable convention. - def timestampToString(us: SQLTimestamp, timeZone: TimeZone): String = { + def timestampToString(tf: TimestampFormatter, us: SQLTimestamp): String = { val ts = toJavaTimestamp(us) val timestampString = ts.toString - val timestampFormat = getThreadLocalTimestampFormat(timeZone) - val formatted = timestampFormat.format(ts) + val formatted = tf.format(us) if (timestampString.length > 19 && timestampString.substring(19) != ".0") { formatted + timestampString.substring(19) @@ -175,27 +110,6 @@ object DateTimeUtils { } } - @tailrec - def stringToTime(s: String): java.util.Date = { - val indexOfGMT = s.indexOf("GMT") - if (indexOfGMT != -1) { - // ISO8601 with a weird time zone specifier (2000-01-01T00:00GMT+01:00) - val s0 = s.substring(0, indexOfGMT) - val s1 = s.substring(indexOfGMT + 3) - // Mapped to 2000-01-01T00:00+01:00 - stringToTime(s0 + s1) - } else if (!s.contains('T')) { - // JDBC escape string - if (s.contains(' ')) { - Timestamp.valueOf(s) - } else { - Date.valueOf(s) - } - } else { - DatatypeConverter.parseDateTime(s).getTime() - } - } - /** * Returns the number of days since epoch from java.sql.Date. */ @@ -427,23 +341,35 @@ object DateTimeUtils { return None } - val c = if (tz.isEmpty) { - Calendar.getInstance(timeZone) + val zoneId = if (tz.isEmpty) { + timeZone.toZoneId } else { - Calendar.getInstance( - getTimeZone(f"GMT${tz.get.toChar}${segments(7)}%02d:${segments(8)}%02d")) + getTimeZone(f"GMT${tz.get.toChar}${segments(7)}%02d:${segments(8)}%02d").toZoneId } - c.set(Calendar.MILLISECOND, 0) - - if (justTime) { - c.set(Calendar.HOUR_OF_DAY, segments(3)) - c.set(Calendar.MINUTE, segments(4)) - c.set(Calendar.SECOND, segments(5)) + val nanoseconds = TimeUnit.MICROSECONDS.toNanos(segments(6)) + val localTime = LocalTime.of(segments(3), segments(4), segments(5), nanoseconds.toInt) + val localDate = if (justTime) { + LocalDate.now(zoneId) } else { - c.set(segments(0), segments(1) - 1, segments(2), segments(3), segments(4), segments(5)) + LocalDate.of(segments(0), segments(1), segments(2)) } + val localDateTime = LocalDateTime.of(localDate, localTime) + val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId) + val instant = Instant.from(zonedDateTime) - Some(c.getTimeInMillis * 1000 + segments(6)) + Some(instantToMicros(instant)) + } + + def instantToMicros(instant: Instant): Long = { + val sec = Math.multiplyExact(instant.getEpochSecond, MICROS_PER_SECOND) + val result = Math.addExact(sec, instant.getNano / NANOS_PER_MICROS) + result + } + + def instantToDays(instant: Instant): Int = { + val seconds = instant.getEpochSecond + val days = Math.floorDiv(seconds, SECONDS_PER_DAY) + days.toInt } /** @@ -496,11 +422,9 @@ object DateTimeUtils { return None } - val c = threadLocalGmtCalendar.get() - c.clear() - c.set(segments(0), segments(1) - 1, segments(2), 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - Some((c.getTimeInMillis / MILLIS_PER_DAY).toInt) + val localDate = LocalDate.of(segments(0), segments(1), segments(2)) + val instant = localDate.atStartOfDay(TimeZoneUTC.toZoneId).toInstant + Some(instantToDays(instant)) } /** @@ -511,9 +435,9 @@ object DateTimeUtils { return true } if (month == 2) { - if (isLeapYear(year) && day > 29) { + if (isLeap(year) && day > 29) { return true - } else if (!isLeapYear(year) && day > 28) { + } else if (!isLeap(year) && day > 28) { return true } } else if (!MonthOf31Days.contains(month) && day > 30) { @@ -583,78 +507,12 @@ object DateTimeUtils { ((localTimestamp(microsec, timeZone) / MICROS_PER_SECOND) % 60).toInt } - private[this] def isLeapYear(year: Int): Boolean = { - (year % 4) == 0 && ((year % 100) != 0 || (year % 400) == 0) - } - - /** - * Return the number of days since the start of 400 year period. - * The second year of a 400 year period (year 1) starts on day 365. - */ - private[this] def yearBoundary(year: Int, isGregorian: Boolean): Int = { - if (isGregorian) { - year * 365 + ((year / 4) - (year / 100) + (year / 400)) - } else { - year * 365 + (year / 4) - } - } - - /** - * Calculates the number of years for the given number of days. This depends - * on a 400 year period. - * @param days days since the beginning of the 400 year period - * @param isGregorian indicates whether leap years should be calculated according to Gregorian - * (or Julian) calendar - * @return (number of year, days in year) - */ - private[this] def numYears(days: Int, isGregorian: Boolean): (Int, Int) = { - val year = days / 365 - val boundary = yearBoundary(year, isGregorian) - if (days > boundary) { - (year, days - boundary) - } else { - (year - 1, days - yearBoundary(year - 1, isGregorian)) - } - } - - /** - * Calculates the year and the number of the day in the year for the given - * number of days. The given days is the number of days since 1.1.1970. - * - * The calculation uses the fact that the period 1.1.2001 until 31.12.2400 is - * equals to the period 1.1.1601 until 31.12.2000. - */ - private[this] def getYearAndDayInYear(daysSince1970: SQLDate): (Int, Int) = { - // Since Julian calendar was replaced with the Gregorian calendar, - // the 10 days after Oct. 4 were skipped. - // (1582-10-04) -141428 days since 1970-01-01 - if (daysSince1970 <= -141428) { - getYearAndDayInYear(daysSince1970 - 10, toYearZeroInJulian, daysIn400YearsInJulian, false) - } else { - getYearAndDayInYear(daysSince1970, toYearZero, daysIn400Years, true) - } - } - - private def getYearAndDayInYear( - daysSince1970: SQLDate, - toYearZero: SQLDate, - daysIn400Years: SQLDate, - isGregorian: Boolean): (Int, Int) = { - // add the difference (in days) between 1.1.1970 and the artificial year 0 (-17999) - val daysNormalized = daysSince1970 + toYearZero - val numOfQuarterCenturies = daysNormalized / daysIn400Years - val daysInThis400 = daysNormalized % daysIn400Years + 1 - val (years, dayInYear) = numYears(daysInThis400, isGregorian) - val year: Int = (2001 - 20000) + 400 * numOfQuarterCenturies + years - (year, dayInYear) - } - /** * Returns the 'day in year' value for the given date. The date is expressed in days * since 1.1.1970. */ def getDayInYear(date: SQLDate): Int = { - getYearAndDayInYear(date)._2 + LocalDate.ofEpochDay(date).getDayOfYear } /** @@ -662,7 +520,7 @@ object DateTimeUtils { * since 1.1.1970. */ def getYear(date: SQLDate): Int = { - getYearAndDayInYear(date)._1 + LocalDate.ofEpochDay(date).getYear } /** @@ -670,19 +528,7 @@ object DateTimeUtils { * since 1.1.1970. */ def getQuarter(date: SQLDate): Int = { - var (year, dayInYear) = getYearAndDayInYear(date) - if (isLeapYear(year)) { - dayInYear = dayInYear - 1 - } - if (dayInYear <= 90) { - 1 - } else if (dayInYear <= 181) { - 2 - } else if (dayInYear <= 273) { - 3 - } else { - 4 - } + LocalDate.ofEpochDay(date).get(IsoFields.QUARTER_OF_YEAR) } /** @@ -690,43 +536,8 @@ object DateTimeUtils { * year, month (Jan is Month 1), dayInMonth, daysToMonthEnd (0 if it's last day of month). */ def splitDate(date: SQLDate): (Int, Int, Int, Int) = { - var (year, dayInYear) = getYearAndDayInYear(date) - val isLeap = isLeapYear(year) - if (isLeap && dayInYear == 60) { - (year, 2, 29, 0) - } else { - if (isLeap && dayInYear > 60) dayInYear -= 1 - - if (dayInYear <= 181) { - if (dayInYear <= 31) { - (year, 1, dayInYear, 31 - dayInYear) - } else if (dayInYear <= 59) { - (year, 2, dayInYear - 31, if (isLeap) 60 - dayInYear else 59 - dayInYear) - } else if (dayInYear <= 90) { - (year, 3, dayInYear - 59, 90 - dayInYear) - } else if (dayInYear <= 120) { - (year, 4, dayInYear - 90, 120 - dayInYear) - } else if (dayInYear <= 151) { - (year, 5, dayInYear - 120, 151 - dayInYear) - } else { - (year, 6, dayInYear - 151, 181 - dayInYear) - } - } else { - if (dayInYear <= 212) { - (year, 7, dayInYear - 181, 212 - dayInYear) - } else if (dayInYear <= 243) { - (year, 8, dayInYear - 212, 243 - dayInYear) - } else if (dayInYear <= 273) { - (year, 9, dayInYear - 243, 273 - dayInYear) - } else if (dayInYear <= 304) { - (year, 10, dayInYear - 273, 304 - dayInYear) - } else if (dayInYear <= 334) { - (year, 11, dayInYear - 304, 334 - dayInYear) - } else { - (year, 12, dayInYear - 334, 365 - dayInYear) - } - } - } + val ld = LocalDate.ofEpochDay(date) + (ld.getYear, ld.getMonthValue, ld.getDayOfMonth, ld.lengthOfMonth() - ld.getDayOfMonth) } /** @@ -734,40 +545,7 @@ object DateTimeUtils { * since 1.1.1970. January is month 1. */ def getMonth(date: SQLDate): Int = { - var (year, dayInYear) = getYearAndDayInYear(date) - if (isLeapYear(year)) { - if (dayInYear == 60) { - return 2 - } else if (dayInYear > 60) { - dayInYear = dayInYear - 1 - } - } - - if (dayInYear <= 31) { - 1 - } else if (dayInYear <= 59) { - 2 - } else if (dayInYear <= 90) { - 3 - } else if (dayInYear <= 120) { - 4 - } else if (dayInYear <= 151) { - 5 - } else if (dayInYear <= 181) { - 6 - } else if (dayInYear <= 212) { - 7 - } else if (dayInYear <= 243) { - 8 - } else if (dayInYear <= 273) { - 9 - } else if (dayInYear <= 304) { - 10 - } else if (dayInYear <= 334) { - 11 - } else { - 12 - } + LocalDate.ofEpochDay(date).getMonthValue } /** @@ -775,40 +553,7 @@ object DateTimeUtils { * since 1.1.1970. */ def getDayOfMonth(date: SQLDate): Int = { - var (year, dayInYear) = getYearAndDayInYear(date) - if (isLeapYear(year)) { - if (dayInYear == 60) { - return 29 - } else if (dayInYear > 60) { - dayInYear = dayInYear - 1 - } - } - - if (dayInYear <= 31) { - dayInYear - } else if (dayInYear <= 59) { - dayInYear - 31 - } else if (dayInYear <= 90) { - dayInYear - 59 - } else if (dayInYear <= 120) { - dayInYear - 90 - } else if (dayInYear <= 151) { - dayInYear - 120 - } else if (dayInYear <= 181) { - dayInYear - 151 - } else if (dayInYear <= 212) { - dayInYear - 181 - } else if (dayInYear <= 243) { - dayInYear - 212 - } else if (dayInYear <= 273) { - dayInYear - 243 - } else if (dayInYear <= 304) { - dayInYear - 273 - } else if (dayInYear <= 334) { - dayInYear - 304 - } else { - dayInYear - 334 - } + LocalDate.ofEpochDay(date).getDayOfMonth } /** @@ -824,7 +569,7 @@ object DateTimeUtils { val absoluteYear = absoluteMonth / 12 var monthInYear = absoluteMonth - absoluteYear * 12 var date = getDateFromYear(absoluteYear) - if (monthInYear >= 2 && isLeapYear(absoluteYear + YearZero)) { + if (monthInYear >= 2 && isLeap(absoluteYear + YearZero)) { date += 1 } while (monthInYear > 0) { @@ -855,7 +600,7 @@ object DateTimeUtils { val currentMonthInYear = nonNegativeMonth % 12 val currentYear = nonNegativeMonth / 12 - val leapDay = if (currentMonthInYear == 1 && isLeapYear(currentYear + YearZero)) 1 else 0 + val leapDay = if (currentMonthInYear == 1 && isLeap(currentYear + YearZero)) 1 else 0 val lastDayOfMonth = monthDays(currentMonthInYear) + leapDay val currentDayInMonth = if (daysToMonthEnd == 0 || dayOfMonth >= lastDayOfMonth) { @@ -977,8 +722,8 @@ object DateTimeUtils { * since 1.1.1970. */ def getLastDayOfMonth(date: SQLDate): SQLDate = { - val (_, _, _, daysToMonthEnd) = splitDate(date) - date + daysToMonthEnd + val localDate = LocalDate.ofEpochDay(date) + (date - localDate.getDayOfMonth) + localDate.lengthOfMonth() } // Visible for testing. @@ -1162,13 +907,4 @@ object DateTimeUtils { def toUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { convertTz(time, getTimeZone(timeZone), TimeZoneGMT) } - - /** - * Re-initialize the current thread's thread locals. Exposed for testing. - */ - private[util] def resetThreadLocals(): Unit = { - threadLocalGmtCalendar.remove() - threadLocalTimestampFormat.remove() - threadLocalDateFormat.remove() - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 8042099e5a92..4ec61e1ca4a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -23,11 +23,7 @@ import java.time.format.DateTimeParseException import java.time.temporal.TemporalQueries import java.util.{Locale, TimeZone} -import scala.util.Try - -import org.apache.commons.lang3.time.FastDateFormat - -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros sealed trait TimestampFormatter extends Serializable { /** @@ -42,7 +38,7 @@ sealed trait TimestampFormatter extends Serializable { @throws(classOf[ParseException]) @throws(classOf[DateTimeParseException]) @throws(classOf[DateTimeException]) - def parse(s: String): Long // returns microseconds since epoch + def parse(s: String): Long def format(us: Long): String } @@ -62,12 +58,6 @@ class Iso8601TimestampFormatter( } } - private def instantToMicros(instant: Instant): Long = { - val sec = Math.multiplyExact(instant.getEpochSecond, DateTimeUtils.MICROS_PER_SECOND) - val result = Math.addExact(sec, instant.getNano / DateTimeUtils.NANOS_PER_MICROS) - result - } - override def parse(s: String): Long = instantToMicros(toInstant(s)) override def format(us: Long): String = { @@ -79,37 +69,19 @@ class Iso8601TimestampFormatter( } } -class LegacyTimestampFormatter( - pattern: String, - timeZone: TimeZone, - locale: Locale) extends TimestampFormatter { - @transient - private lazy val format = FastDateFormat.getInstance(pattern, timeZone, locale) - - protected def toMillis(s: String): Long = format.parse(s).getTime - - override def parse(s: String): Long = toMillis(s) * DateTimeUtils.MICROS_PER_MILLIS +object TimestampFormatter { + val defaultPattern: String = "yyyy-MM-dd HH:mm:ss" + val defaultLocale: Locale = Locale.US - override def format(us: Long): String = { - format.format(DateTimeUtils.toJavaTimestamp(us)) + def apply(format: String, timeZone: TimeZone, locale: Locale): TimestampFormatter = { + new Iso8601TimestampFormatter(format, timeZone, locale) } -} -class LegacyFallbackTimestampFormatter( - pattern: String, - timeZone: TimeZone, - locale: Locale) extends LegacyTimestampFormatter(pattern, timeZone, locale) { - override def toMillis(s: String): Long = { - Try {super.toMillis(s)}.getOrElse(DateTimeUtils.stringToTime(s).getTime) + def apply(format: String, timeZone: TimeZone): TimestampFormatter = { + apply(format, timeZone, defaultLocale) } -} -object TimestampFormatter { - def apply(format: String, timeZone: TimeZone, locale: Locale): TimestampFormatter = { - if (SQLConf.get.legacyTimeParserEnabled) { - new LegacyFallbackTimestampFormatter(format, timeZone, locale) - } else { - new Iso8601TimestampFormatter(format, timeZone, locale) - } + def apply(timeZone: TimeZone): TimestampFormatter = { + apply(defaultPattern, timeZone, defaultLocale) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9804af7dff17..ebc8c3705ea2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1419,8 +1419,15 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(100) - val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers") + val USE_V1_SOURCE_READER_LIST = buildConf("spark.sql.sources.read.useV1SourceList") .internal() + .doc("A comma-separated list of data source short names or fully qualified data source" + + " register class names for which data source V2 read paths are disabled. Reads from these" + + " sources will fall back to the V1 sources.") + .stringConf + .createWithDefault("") + + val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers") .doc("A comma-separated list of fully qualified data source register class names for which" + " StreamWriteSupport is disabled. Writes to these sources will fall back to the V1 Sinks.") .stringConf @@ -1625,13 +1632,6 @@ object SQLConf { "a SparkConf entry.") .booleanConf .createWithDefault(true) - - val LEGACY_TIME_PARSER_ENABLED = buildConf("spark.sql.legacy.timeParser.enabled") - .doc("When set to true, java.text.SimpleDateFormat is used for formatting and parsing " + - " dates/timestamps in a locale-sensitive manner. When set to false, classes from " + - "java.time.* packages are used for the same purpose.") - .booleanConf - .createWithDefault(false) } /** @@ -2009,6 +2009,8 @@ class SQLConf extends Serializable with Logging { def continuousStreamingExecutorPollIntervalMs: Long = getConf(CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS) + def userV1SourceReaderList: String = getConf(USE_V1_SOURCE_READER_LIST) + def disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS) def disabledV2StreamingMicroBatchReaders: String = @@ -2057,8 +2059,6 @@ class SQLConf extends Serializable with Logging { def setCommandRejectsSparkCoreConfs: Boolean = getConf(SQLConf.SET_COMMAND_REJECTS_SPARK_CORE_CONFS) - def legacyTimeParserEnabled: Boolean = getConf(SQLConf.LEGACY_TIME_PARSER_ENABLED) - /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 0a8dc2835ea4..582af9fb29be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -132,4 +132,15 @@ object StaticSQLConf { .intConf .createWithDefault(1000) + val BROADCAST_EXCHANGE_MAX_THREAD_THREASHOLD = + buildStaticConf("spark.sql.broadcastExchange.maxThreadNumber") + .doc("The maximum degree of parallelism to fetch and broadcast the table." + + "If we encounter memory issue like frequently full GC or OOM when broadcast table " + + "we can decrease this number in order to reduce memory usage." + + "Notice the number should be carefully chosen since decreasing parallelism might " + + "cause longer waiting for other broadcasting. Also, increasing parallelism may " + + "cause memory problem.") + .intConf + .checkValue(thres => thres > 0, "The threshold should be positive.") + .createWithDefault(128) } diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java index 8da778800bb9..16452b4bd538 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java @@ -29,6 +29,7 @@ import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.internal.config.package$; import java.util.Random; @@ -105,8 +106,8 @@ private boolean checkValue(UnsafeRow row, long v1, long v2) { public void setup() { memoryManager = new TestMemoryManager(new SparkConf() .set("spark.memory.offHeap.enabled", "false") - .set("spark.shuffle.spill.compress", "false") - .set("spark.shuffle.compress", "false")); + .set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false) + .set(package$.MODULE$.SHUFFLE_COMPRESS(), false)); taskMemoryManager = new TaskMemoryManager(memoryManager, 0); } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index c9d733726ff2..c3b7e19455cd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -20,11 +20,12 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.{Calendar, Locale, TimeZone} +import java.util.concurrent.TimeUnit import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -40,12 +41,14 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val pstId = Option(TimeZonePST.getID) val jstId = Option(TimeZoneJST.getID) - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) - sdf.setTimeZone(TimeZoneGMT) - val sdfDate = new SimpleDateFormat("yyyy-MM-dd", Locale.US) - sdfDate.setTimeZone(TimeZoneGMT) - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) + def toMillis(timestamp: String): Long = { + val tf = TimestampFormatter("yyyy-MM-dd HH:mm:ss", TimeZoneGMT) + TimeUnit.MICROSECONDS.toMillis(tf.parse(timestamp)) + } + val date = "2015-04-08 13:10:15" + val d = new Date(toMillis(date)) + val time = "2013-11-08 13:10:15" + val ts = new Timestamp(toMillis(time)) test("datetime function current_date") { val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT) @@ -78,15 +81,15 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } checkEvaluation(DayOfYear(Literal.create(null, DateType)), null) - checkEvaluation(DayOfYear(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 288) - checkEvaluation(DayOfYear(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 277) + checkEvaluation(DayOfYear(Cast(Literal("1582-10-15 13:10:15"), DateType)), 288) + checkEvaluation(DayOfYear(Cast(Literal("1582-10-04 13:10:15"), DateType)), 277) checkConsistencyBetweenInterpretedAndCodegen(DayOfYear, DateType) } test("Year") { checkEvaluation(Year(Literal.create(null, DateType)), null) checkEvaluation(Year(Literal(d)), 2015) - checkEvaluation(Year(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 2015) + checkEvaluation(Year(Cast(Literal(date), DateType, gmtId)), 2015) checkEvaluation(Year(Cast(Literal(ts), DateType, gmtId)), 2013) val c = Calendar.getInstance() @@ -100,15 +103,15 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } } - checkEvaluation(Year(Literal(new Date(sdf.parse("1582-01-01 13:10:15").getTime))), 1582) - checkEvaluation(Year(Literal(new Date(sdf.parse("1581-12-31 13:10:15").getTime))), 1581) + checkEvaluation(Year(Cast(Literal("1582-01-01 13:10:15"), DateType)), 1582) + checkEvaluation(Year(Cast(Literal("1581-12-31 13:10:15"), DateType)), 1581) checkConsistencyBetweenInterpretedAndCodegen(Year, DateType) } test("Quarter") { checkEvaluation(Quarter(Literal.create(null, DateType)), null) checkEvaluation(Quarter(Literal(d)), 2) - checkEvaluation(Quarter(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 2) + checkEvaluation(Quarter(Cast(Literal(date), DateType, gmtId)), 2) checkEvaluation(Quarter(Cast(Literal(ts), DateType, gmtId)), 4) val c = Calendar.getInstance() @@ -123,20 +126,20 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } - checkEvaluation(Quarter(Literal(new Date(sdf.parse("1582-10-01 13:10:15").getTime))), 4) - checkEvaluation(Quarter(Literal(new Date(sdf.parse("1582-09-30 13:10:15").getTime))), 3) + checkEvaluation(Quarter(Cast(Literal("1582-10-01 13:10:15"), DateType)), 4) + checkEvaluation(Quarter(Cast(Literal("1582-09-30 13:10:15"), DateType)), 3) checkConsistencyBetweenInterpretedAndCodegen(Quarter, DateType) } test("Month") { checkEvaluation(Month(Literal.create(null, DateType)), null) checkEvaluation(Month(Literal(d)), 4) - checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 4) + checkEvaluation(Month(Cast(Literal(date), DateType, gmtId)), 4) checkEvaluation(Month(Cast(Literal(ts), DateType, gmtId)), 11) - checkEvaluation(Month(Literal(new Date(sdf.parse("1582-04-28 13:10:15").getTime))), 4) - checkEvaluation(Month(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 10) - checkEvaluation(Month(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 10) + checkEvaluation(Month(Cast(Literal("1582-04-28 13:10:15"), DateType)), 4) + checkEvaluation(Month(Cast(Literal("1582-10-04 13:10:15"), DateType)), 10) + checkEvaluation(Month(Cast(Literal("1582-10-15 13:10:15"), DateType)), 10) val c = Calendar.getInstance() (2003 to 2004).foreach { y => @@ -156,12 +159,12 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(DayOfMonth(Cast(Literal("2000-02-29"), DateType)), 29) checkEvaluation(DayOfMonth(Literal.create(null, DateType)), null) checkEvaluation(DayOfMonth(Literal(d)), 8) - checkEvaluation(DayOfMonth(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 8) + checkEvaluation(DayOfMonth(Cast(Literal(date), DateType, gmtId)), 8) checkEvaluation(DayOfMonth(Cast(Literal(ts), DateType, gmtId)), 8) - checkEvaluation(DayOfMonth(Literal(new Date(sdf.parse("1582-04-28 13:10:15").getTime))), 28) - checkEvaluation(DayOfMonth(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 15) - checkEvaluation(DayOfMonth(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 4) + checkEvaluation(DayOfMonth(Cast(Literal("1582-04-28 13:10:15"), DateType)), 28) + checkEvaluation(DayOfMonth(Cast(Literal("1582-10-15 13:10:15"), DateType)), 15) + checkEvaluation(DayOfMonth(Cast(Literal("1582-10-04 13:10:15"), DateType)), 4) val c = Calendar.getInstance() (1999 to 2000).foreach { y => @@ -179,7 +182,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(Second(Literal.create(null, DateType), gmtId).resolved === false) assert(Second(Cast(Literal(d), TimestampType, gmtId), gmtId).resolved === true) checkEvaluation(Second(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) - checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType, gmtId), gmtId), 15) + checkEvaluation(Second(Cast(Literal(date), TimestampType, gmtId), gmtId), 15) checkEvaluation(Second(Literal(ts), gmtId), 15) val c = Calendar.getInstance() @@ -200,13 +203,13 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("DayOfWeek") { checkEvaluation(DayOfWeek(Literal.create(null, DateType)), null) checkEvaluation(DayOfWeek(Literal(d)), Calendar.WEDNESDAY) - checkEvaluation(DayOfWeek(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), + checkEvaluation(DayOfWeek(Cast(Literal(date), DateType, gmtId)), Calendar.WEDNESDAY) checkEvaluation(DayOfWeek(Cast(Literal(ts), DateType, gmtId)), Calendar.FRIDAY) checkEvaluation(DayOfWeek(Cast(Literal("2011-05-06"), DateType, gmtId)), Calendar.FRIDAY) - checkEvaluation(DayOfWeek(Literal(new Date(sdf.parse("2017-05-27 13:10:15").getTime))), + checkEvaluation(DayOfWeek(Literal(new Date(toMillis("2017-05-27 13:10:15")))), Calendar.SATURDAY) - checkEvaluation(DayOfWeek(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), + checkEvaluation(DayOfWeek(Literal(new Date(toMillis("1582-10-15 13:10:15")))), Calendar.FRIDAY) checkConsistencyBetweenInterpretedAndCodegen(DayOfWeek, DateType) } @@ -214,22 +217,22 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("WeekDay") { checkEvaluation(WeekDay(Literal.create(null, DateType)), null) checkEvaluation(WeekDay(Literal(d)), 2) - checkEvaluation(WeekDay(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 2) + checkEvaluation(WeekDay(Cast(Literal(date), DateType, gmtId)), 2) checkEvaluation(WeekDay(Cast(Literal(ts), DateType, gmtId)), 4) checkEvaluation(WeekDay(Cast(Literal("2011-05-06"), DateType, gmtId)), 4) - checkEvaluation(WeekDay(Literal(new Date(sdf.parse("2017-05-27 13:10:15").getTime))), 5) - checkEvaluation(WeekDay(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 4) + checkEvaluation(WeekDay(Literal(new Date(toMillis("2017-05-27 13:10:15")))), 5) + checkEvaluation(WeekDay(Literal(new Date(toMillis("1582-10-15 13:10:15")))), 4) checkConsistencyBetweenInterpretedAndCodegen(WeekDay, DateType) } test("WeekOfYear") { checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) checkEvaluation(WeekOfYear(Literal(d)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(date), DateType, gmtId)), 15) checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType, gmtId)), 45) checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType, gmtId)), 18) - checkEvaluation(WeekOfYear(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 40) - checkEvaluation(WeekOfYear(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 40) + checkEvaluation(WeekOfYear(Literal(new Date(toMillis("1582-10-15 13:10:15")))), 40) + checkEvaluation(WeekOfYear(Literal(new Date(toMillis("1582-10-04 13:10:15")))), 39) checkConsistencyBetweenInterpretedAndCodegen(WeekOfYear, DateType) } @@ -266,7 +269,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(Hour(Literal.create(null, DateType), gmtId).resolved === false) assert(Hour(Literal(ts), gmtId).resolved === true) checkEvaluation(Hour(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) - checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType, gmtId), gmtId), 13) + checkEvaluation(Hour(Cast(Literal(date), TimestampType, gmtId), gmtId), 13) checkEvaluation(Hour(Literal(ts), gmtId), 13) val c = Calendar.getInstance() @@ -293,7 +296,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(Minute(Literal(ts), gmtId).resolved === true) checkEvaluation(Minute(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) checkEvaluation( - Minute(Cast(Literal(sdf.format(d)), TimestampType, gmtId), gmtId), 10) + Minute(Cast(Literal(date), TimestampType, gmtId), gmtId), 10) checkEvaluation(Minute(Literal(ts), gmtId), 10) val c = Calendar.getInstance() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index 4281c89ac475..555ccb892497 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -183,7 +183,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkHiveHashForDateType("2017-01-01", 17167) // boundary cases - checkHiveHashForDateType("0000-01-01", -719530) + checkHiveHashForDateType("0000-01-01", -719528) checkHiveHashForDateType("9999-12-31", 2932896) // epoch @@ -226,7 +226,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { TimeZone.getTimeZone("US/Pacific")) // boundary cases - checkHiveHashForTimestampType("0001-01-01 00:00:00", 1645926784) + checkHiveHashForTimestampType("0001-01-01 00:00:00", 1645969984) checkHiveHashForTimestampType("9999-01-01 00:00:00", -1081818240) // epoch diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index 238e6e34b4ae..b190d6f5caa1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -548,7 +548,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( JsonToStructs(schema, Map.empty, Literal.create(" ", StringType), gmtId), - InternalRow(null) + null ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala index 9a6f4f5f9b0c..8ce45f06ba65 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala @@ -21,7 +21,6 @@ import com.fasterxml.jackson.core.JsonFactory import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { @@ -43,61 +42,45 @@ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("inferring timestamp type") { - Seq(true, false).foreach { legacyParser => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { - checkTimestampType("yyyy", """{"a": "2018"}""") - checkTimestampType("yyyy=MM", """{"a": "2018=12"}""") - checkTimestampType("yyyy MM dd", """{"a": "2018 12 02"}""") - checkTimestampType( - "yyyy-MM-dd'T'HH:mm:ss.SSS", - """{"a": "2018-12-02T21:04:00.123"}""") - checkTimestampType( - "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXX", - """{"a": "2018-12-02T21:04:00.123567+01:00"}""") - } - } + checkTimestampType("yyyy", """{"a": "2018"}""") + checkTimestampType("yyyy=MM", """{"a": "2018=12"}""") + checkTimestampType("yyyy MM dd", """{"a": "2018 12 02"}""") + checkTimestampType( + "yyyy-MM-dd'T'HH:mm:ss.SSS", + """{"a": "2018-12-02T21:04:00.123"}""") + checkTimestampType( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXX", + """{"a": "2018-12-02T21:04:00.123567+01:00"}""") } test("prefer decimals over timestamps") { - Seq(true, false).foreach { legacyParser => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { - checkType( - options = Map( - "prefersDecimal" -> "true", - "timestampFormat" -> "yyyyMMdd.HHmmssSSS" - ), - json = """{"a": "20181202.210400123"}""", - dt = DecimalType(17, 9) - ) - } - } + checkType( + options = Map( + "prefersDecimal" -> "true", + "timestampFormat" -> "yyyyMMdd.HHmmssSSS" + ), + json = """{"a": "20181202.210400123"}""", + dt = DecimalType(17, 9) + ) } test("skip decimal type inferring") { - Seq(true, false).foreach { legacyParser => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { - checkType( - options = Map( - "prefersDecimal" -> "false", - "timestampFormat" -> "yyyyMMdd.HHmmssSSS" - ), - json = """{"a": "20181202.210400123"}""", - dt = TimestampType - ) - } - } + checkType( + options = Map( + "prefersDecimal" -> "false", + "timestampFormat" -> "yyyyMMdd.HHmmssSSS" + ), + json = """{"a": "20181202.210400123"}""", + dt = TimestampType + ) } test("fallback to string type") { - Seq(true, false).foreach { legacyParser => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { - checkType( - options = Map("timestampFormat" -> "yyyy,MM,dd.HHmmssSSS"), - json = """{"a": "20181202.210400123"}""", - dt = StringType - ) - } - } + checkType( + options = Map("timestampFormat" -> "yyyy,MM,dd.HHmmssSSS"), + json = """{"a": "20181202.210400123"}""", + dt = StringType + ) } test("disable timestamp inferring") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala index 0cd6e092e203..73112e3d3bb4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala @@ -400,5 +400,14 @@ class ColumnPruningSuite extends PlanTest { comparePlans(optimized, expected) } + test("SPARK-26619: Prune the unused serializers from SerializeFromObject") { + val testRelation = LocalRelation('_1.int, '_2.int) + val serializerObject = CatalystSerde.serialize[(Int, Int)]( + CatalystSerde.deserialize[(Int, Int)](testRelation)) + val query = serializerObject.select('_1) + val optimized = Optimize.execute(query.analyze) + val expected = serializerObject.copy(serializer = Seq(serializerObject.serializer.head)).analyze + comparePlans(optimized, expected) + } // todo: add more tests for column pruning } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala index 9093d7fecb0f..c570643c7410 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala @@ -66,7 +66,7 @@ class JoinOptimizationSuite extends PlanTest { def testExtractCheckCross (plan: LogicalPlan, expected: Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression])]) { assert( - ExtractFiltersAndInnerJoins.unapply(plan) === expected.map(e => (e._1, e._2, Map.empty))) + ExtractFiltersAndInnerJoins.unapply(plan) === expected.map(e => (e._1, e._2))) } testExtract(x, None) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala index 0dee84620586..f1da0a8e865b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala @@ -292,77 +292,56 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { assertEqualPlans(originalPlan, bestPlan) } - test("hints preservation") { - // Apply hints if we find an equivalent node in the new plan, otherwise discard them. + test("don't reorder if hints present") { val originalPlan = - t1.join(t2.hint("broadcast")).hint("broadcast").join(t4.join(t3).hint("broadcast")) - .where((nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2")) && - (nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && - (nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) - - val bestPlan = - t1.join(t2.hint("broadcast"), Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) - .hint("broadcast") - .join( - t4.join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) - .hint("broadcast"), - Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .hint("broadcast") + .join( + t4.join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + .hint("broadcast"), + Inner, + Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) - assertEqualPlans(originalPlan, bestPlan) + assertEqualPlans(originalPlan, originalPlan) val originalPlan2 = - t1.join(t2).hint("broadcast").join(t3).hint("broadcast").join(t4.hint("broadcast")) - .where((nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2")) && - (nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && - (nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) - - val bestPlan2 = t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) .hint("broadcast") - .join( - t4.hint("broadcast") - .join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))), - Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) - .select(outputsOf(t1, t2, t3, t4): _*) + .join(t4, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + .hint("broadcast") + .join(t3, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) - assertEqualPlans(originalPlan2, bestPlan2) + assertEqualPlans(originalPlan2, originalPlan2) + } - val originalPlan3 = - t1.join(t4).hint("broadcast") - .join(t2.hint("broadcast")).hint("broadcast") - .join(t3.hint("broadcast")) - .where((nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2")) && - (nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && - (nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + test("reorder below and above the hint node") { + val originalPlan = + t1.join(t2).join(t3) + .where((nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && + (nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .hint("broadcast").join(t4) - val bestPlan3 = - t1.join(t2.hint("broadcast"), Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) - .join( - t4.join(t3.hint("broadcast"), - Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))), - Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) - .select(outputsOf(t1, t4, t2, t3): _*) + val bestPlan = + t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) + .hint("broadcast").join(t4) - assertEqualPlans(originalPlan3, bestPlan3) + assertEqualPlans(originalPlan, bestPlan) - val originalPlan4 = - t2.hint("broadcast") - .join(t4).hint("broadcast") - .join(t3.hint("broadcast")).hint("broadcast") - .join(t1) - .where((nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2")) && - (nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && - (nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + val originalPlan2 = + t1.join(t2).join(t3) + .where((nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && + (nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t4.hint("broadcast")) - val bestPlan4 = - t1.join(t2.hint("broadcast"), Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) - .join( - t4.join(t3.hint("broadcast"), - Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))), - Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) - .select(outputsOf(t2, t4, t3, t1): _*) + val bestPlan2 = + t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) + .join(t4.hint("broadcast")) - assertEqualPlans(originalPlan4, bestPlan4) + assertEqualPlans(originalPlan2, bestPlan2) } private def assertEqualPlans( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 8bcc69d580d8..7541d9d089b2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.sql.catalyst.parser import java.sql.{Date, Timestamp} +import java.time.LocalDateTime +import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -56,6 +59,13 @@ class ExpressionParserSuite extends PlanTest { } } + def assertEval( + sqlCommand: String, + expect: Any, + parser: ParserInterface = defaultParser): Unit = { + assert(parser.parseExpression(sqlCommand).eval() === expect) + } + test("star expressions") { // Global Star assertEqual("*", UnresolvedStar(None)) @@ -680,4 +690,36 @@ class ExpressionParserSuite extends PlanTest { assertEqual("last(a ignore nulls)", Last('a, Literal(true)).toAggregateExpression()) assertEqual("last(a)", Last('a, Literal(false)).toAggregateExpression()) } + + test("timestamp literals") { + DateTimeTestUtils.outstandingTimezones.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone.getID) { + def toMicros(time: LocalDateTime): Long = { + val seconds = time.atZone(timeZone.toZoneId).toInstant.getEpochSecond + TimeUnit.SECONDS.toMicros(seconds) + } + assertEval( + sqlCommand = "TIMESTAMP '2019-01-14 20:54:00.000'", + expect = toMicros(LocalDateTime.of(2019, 1, 14, 20, 54))) + assertEval( + sqlCommand = "Timestamp '2000-01-01T00:55:00'", + expect = toMicros(LocalDateTime.of(2000, 1, 1, 0, 55))) + // Parsing of the string does not depend on the SQL config because the string contains + // time zone offset already. + assertEval( + sqlCommand = "TIMESTAMP '2019-01-16 20:50:00.567000+01:00'", + expect = 1547668200567000L) + } + } + } + + test("date literals") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + assertEval("DATE '2019-01-14'", 17910) + assertEval("DATE '2019-01'", 17897) + assertEval("DATE '2019'", 17897) + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala index 66d8d28988f8..442d78598a38 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala @@ -40,12 +40,10 @@ object DateTimeTestUtils { def withDefaultTimeZone[T](newDefaultTimeZone: TimeZone)(block: => T): T = { val originalDefaultTimeZone = TimeZone.getDefault try { - DateTimeUtils.resetThreadLocals() TimeZone.setDefault(newDefaultTimeZone) block } finally { TimeZone.setDefault(originalDefaultTimeZone) - DateTimeUtils.resetThreadLocals() } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 2cb6110e2c09..ef34150fa623 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat +import java.time.LocalDate import java.util.{Calendar, Locale, TimeZone} +import java.util.concurrent.TimeUnit import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeUtils._ @@ -29,16 +31,17 @@ class DateTimeUtilsSuite extends SparkFunSuite { val TimeZonePST = TimeZone.getTimeZone("PST") - private[this] def getInUTCDays(timestamp: Long): Int = { - val tz = TimeZone.getDefault - ((timestamp + tz.getOffset(timestamp)) / MILLIS_PER_DAY).toInt + private[this] def getInUTCDays(localDate: LocalDate): Int = { + val epochSeconds = localDate.atStartOfDay(TimeZoneUTC.toZoneId).toEpochSecond + TimeUnit.SECONDS.toDays(epochSeconds).toInt } test("nanoseconds truncation") { + val tf = TimestampFormatter(DateTimeUtils.defaultTimeZone()) def checkStringToTimestamp(originalTime: String, expectedParsedTime: String) { val parsedTimestampOp = DateTimeUtils.stringToTimestamp(UTF8String.fromString(originalTime)) assert(parsedTimestampOp.isDefined, "timestamp with nanoseconds was not parsed correctly") - assert(DateTimeUtils.timestampToString(parsedTimestampOp.get) === expectedParsedTime) + assert(DateTimeUtils.timestampToString(tf, parsedTimestampOp.get) === expectedParsedTime) } checkStringToTimestamp("2015-01-02 00:00:00.123456789", "2015-01-02 00:00:00.123456") @@ -130,8 +133,10 @@ class DateTimeUtilsSuite extends SparkFunSuite { millisToDays(c.getTimeInMillis)) c.set(1, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) + val localDate = LocalDate.of(1, 1, 1) + .atStartOfDay(TimeZoneUTC.toZoneId) assert(stringToDate(UTF8String.fromString("0001")).get === - millisToDays(c.getTimeInMillis)) + TimeUnit.SECONDS.toDays(localDate.toEpochSecond)) c = Calendar.getInstance() c.set(2015, 2, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) @@ -156,38 +161,6 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(stringToDate(UTF8String.fromString("02015")).isEmpty) } - test("string to time") { - // Tests with UTC. - val c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) - c.set(Calendar.MILLISECOND, 0) - - c.set(1900, 0, 1, 0, 0, 0) - assert(stringToTime("1900-01-01T00:00:00GMT-00:00") === c.getTime()) - - c.set(2000, 11, 30, 10, 0, 0) - assert(stringToTime("2000-12-30T10:00:00Z") === c.getTime()) - - // Tests with set time zone. - c.setTimeZone(TimeZone.getTimeZone("GMT-04:00")) - c.set(Calendar.MILLISECOND, 0) - - c.set(1900, 0, 1, 0, 0, 0) - assert(stringToTime("1900-01-01T00:00:00-04:00") === c.getTime()) - - c.set(1900, 0, 1, 0, 0, 0) - assert(stringToTime("1900-01-01T00:00:00GMT-04:00") === c.getTime()) - - // Tests with local time zone. - c.setTimeZone(TimeZone.getDefault()) - c.set(Calendar.MILLISECOND, 0) - - c.set(2000, 11, 30, 0, 0, 0) - assert(stringToTime("2000-12-30") === new Date(c.getTimeInMillis())) - - c.set(2000, 11, 30, 10, 0, 0) - assert(stringToTime("2000-12-30 10:00:00") === new Timestamp(c.getTimeInMillis())) - } - test("string to timestamp") { for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { def checkStringToTimestamp(str: String, expected: Option[Long]): Unit = { @@ -200,7 +173,9 @@ class DateTimeUtilsSuite extends SparkFunSuite { checkStringToTimestamp("1969-12-31 16:00:00", Option(c.getTimeInMillis * 1000)) c.set(1, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("0001", Option(c.getTimeInMillis * 1000)) + val date = LocalDate.of(1, 1, 1) + .atStartOfDay(tz.toZoneId) + checkStringToTimestamp("0001", Option(TimeUnit.SECONDS.toMicros(date.toEpochSecond))) c = Calendar.getInstance(tz) c.set(2015, 2, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) @@ -403,73 +378,47 @@ class DateTimeUtilsSuite extends SparkFunSuite { } test("get day in year") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 77) - c.set(2012, 2, 18, 0, 0, 0) - assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 78) + assert(getDayInYear(getInUTCDays(LocalDate.of(2015, 3, 18))) === 77) + assert(getDayInYear(getInUTCDays(LocalDate.of(2012, 3, 18))) === 78) } - test("SPARK-26002: correct day of year calculations for Julian calendar years") { - val c = Calendar.getInstance() - c.set(Calendar.MILLISECOND, 0) - (1000 to 1600 by 100).foreach { year => + test("day of year calculations for old years") { + var date = LocalDate.of(1582, 3, 1) + assert(getDayInYear(getInUTCDays(date)) === 60) + + (1000 to 1600 by 10).foreach { year => // January 1 is the 1st day of year. - c.set(year, 0, 1, 0, 0, 0) - assert(getYear(getInUTCDays(c.getTimeInMillis)) === year) - assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 1) - assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 1) - - // March 1 is the 61st day of the year as they are leap years. It is true for - // even the multiples of 100 as before 1582-10-4 the Julian calendar leap year calculation - // is used in which every multiples of 4 are leap years - c.set(year, 2, 1, 0, 0, 0) - assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 61) - assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 3) - - // testing leap day (February 29) in leap years - c.set(year, 1, 29, 0, 0, 0) - assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 60) - - // For non-leap years: - c.set(year + 1, 2, 1, 0, 0, 0) - assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 60) + date = LocalDate.of(year, 1, 1) + assert(getYear(getInUTCDays(date)) === year) + assert(getMonth(getInUTCDays(date)) === 1) + assert(getDayInYear(getInUTCDays(date)) === 1) + + // December 31 is the 1st day of year. + date = LocalDate.of(year, 12, 31) + assert(getYear(getInUTCDays(date)) === year) + assert(getMonth(getInUTCDays(date)) === 12) + assert(getDayOfMonth(getInUTCDays(date)) === 31) } - - c.set(1582, 2, 1, 0, 0, 0) - assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 60) } test("get year") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getYear(getInUTCDays(c.getTimeInMillis)) === 2015) - c.set(2012, 2, 18, 0, 0, 0) - assert(getYear(getInUTCDays(c.getTimeInMillis)) === 2012) + assert(getYear(getInUTCDays(LocalDate.of(2015, 2, 18))) === 2015) + assert(getYear(getInUTCDays(LocalDate.of(2012, 2, 18))) === 2012) } test("get quarter") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getQuarter(getInUTCDays(c.getTimeInMillis)) === 1) - c.set(2012, 11, 18, 0, 0, 0) - assert(getQuarter(getInUTCDays(c.getTimeInMillis)) === 4) + assert(getQuarter(getInUTCDays(LocalDate.of(2015, 2, 18))) === 1) + assert(getQuarter(getInUTCDays(LocalDate.of(2012, 11, 18))) === 4) } test("get month") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 3) - c.set(2012, 11, 18, 0, 0, 0) - assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 12) + assert(getMonth(getInUTCDays(LocalDate.of(2015, 3, 18))) === 3) + assert(getMonth(getInUTCDays(LocalDate.of(2012, 12, 18))) === 12) } test("get day of month") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 18) - c.set(2012, 11, 24, 0, 0, 0) - assert(getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 24) + assert(getDayOfMonth(getInUTCDays(LocalDate.of(2015, 3, 18))) === 18) + assert(getDayOfMonth(getInUTCDays(LocalDate.of(2012, 12, 24))) === 24) } test("date add months") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala index 2dc55e0e1f63..4d5872c92f5a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala @@ -29,7 +29,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { test("parsing dates") { DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val formatter = DateFormatter() val daysSinceEpoch = formatter.parse("2018-12-02") assert(daysSinceEpoch === 17867) } @@ -39,7 +39,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { test("format dates") { DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val formatter = DateFormatter() val date = formatter.format(17867) assert(date === "2018-12-02") } @@ -59,7 +59,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { "5010-11-17").foreach { date => DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val formatter = DateFormatter() val days = formatter.parse(date) val formatted = formatter.format(days) assert(date === formatted) @@ -82,7 +82,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { 1110657).foreach { days => DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter("yyyy-MM-dd", Locale.US) + val formatter = DateFormatter() val date = formatter.format(days) val parsed = formatter.parse(date) assert(days === parsed) @@ -92,7 +92,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { } test("parsing date without explicit day") { - val formatter = DateFormatter("yyyy MMM", Locale.US) + val formatter = DateFormatter("yyyy MMM") val daysSinceEpoch = formatter.parse("2018 Dec") assert(daysSinceEpoch === LocalDate.of(2018, 12, 1).toEpochDay) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index 2ce3eacc30cc..d007adf3aab8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -41,8 +41,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => val formatter = TimestampFormatter( "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", - TimeZone.getTimeZone(timeZone), - Locale.US) + TimeZone.getTimeZone(timeZone)) val microsSinceEpoch = formatter.parse(localDate) assert(microsSinceEpoch === expectedMicros(timeZone)) } @@ -62,8 +61,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => val formatter = TimestampFormatter( "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", - TimeZone.getTimeZone(timeZone), - Locale.US) + TimeZone.getTimeZone(timeZone)) val timestamp = formatter.format(microsSinceEpoch) assert(timestamp === expectedTimestamp(timeZone)) } @@ -82,7 +80,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { 2177456523456789L, 11858049903010203L).foreach { micros => DateTimeTestUtils.outstandingTimezones.foreach { timeZone => - val formatter = TimestampFormatter(pattern, timeZone, Locale.US) + val formatter = TimestampFormatter(pattern, timeZone) val timestamp = formatter.format(micros) val parsed = formatter.parse(timestamp) assert(micros === parsed) @@ -103,7 +101,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { "2039-01-01T01:02:03.456789", "2345-10-07T22:45:03.010203").foreach { timestamp => DateTimeTestUtils.outstandingTimezones.foreach { timeZone => - val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone, Locale.US) + val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone) val micros = formatter.parse(timestamp) val formatted = formatter.format(micros) assert(timestamp === formatted) @@ -114,8 +112,7 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { test(" case insensitive parsing of am and pm") { val formatter = TimestampFormatter( "yyyy MMM dd hh:mm:ss a", - TimeZone.getTimeZone("UTC"), - Locale.US) + TimeZone.getTimeZone("UTC")) val micros = formatter.parse("2009 Mar 20 11:30:01 am") assert(micros === TimeUnit.SECONDS.toMicros( LocalDateTime.of(2009, 3, 20, 11, 30, 1).toEpochSecond(ZoneOffset.UTC))) diff --git a/sql/core/benchmarks/InExpressionBenchmark-results.txt b/sql/core/benchmarks/InExpressionBenchmark-results.txt new file mode 100644 index 000000000000..d2adbded6614 --- /dev/null +++ b/sql/core/benchmarks/InExpressionBenchmark-results.txt @@ -0,0 +1,551 @@ +================================================================================================ +In Expression Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 bytes: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 101 / 138 98.7 10.1 1.0X +InSet expression 125 / 136 79.7 12.5 0.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 bytes: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 101 / 111 99.3 10.1 1.0X +InSet expression 126 / 133 79.6 12.6 0.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 bytes: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 176 / 183 56.9 17.6 1.0X +InSet expression 174 / 184 57.4 17.4 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 bytes: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 299 / 312 33.5 29.9 1.0X +InSet expression 243 / 246 41.2 24.3 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 bytes: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 512 / 518 19.5 51.2 1.0X +InSet expression 388 / 400 25.8 38.8 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 bytes: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 854 / 866 11.7 85.4 1.0X +InSet expression 686 / 694 14.6 68.6 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 shorts: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 59 / 62 169.6 5.9 1.0X +InSet expression 163 / 168 61.3 16.3 0.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 shorts: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 76 / 78 132.0 7.6 1.0X +InSet expression 182 / 186 54.9 18.2 0.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 shorts: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 126 / 128 79.4 12.6 1.0X +InSet expression 190 / 193 52.7 19.0 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 shorts: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 227 / 227 44.1 22.7 1.0X +InSet expression 232 / 235 43.1 23.2 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 shorts: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 408 / 414 24.5 40.8 1.0X +InSet expression 203 / 209 49.3 20.3 2.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 shorts: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 762 / 765 13.1 76.2 1.0X +InSet expression 192 / 196 52.1 19.2 4.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 53 / 57 187.3 5.3 1.0X +InSet expression 156 / 160 63.9 15.6 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 70 / 74 142.4 7.0 1.0X +InSet expression 170 / 176 58.9 17.0 0.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 125 / 126 80.2 12.5 1.0X +InSet expression 174 / 179 57.4 17.4 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 220 / 222 45.5 22.0 1.0X +InSet expression 215 / 221 46.6 21.5 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 404 / 407 24.8 40.4 1.0X +InSet expression 189 / 192 53.0 18.9 2.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 760 / 764 13.2 76.0 1.0X +InSet expression 176 / 179 56.8 17.6 4.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 50 / 52 200.3 5.0 1.0X +InSet expression 147 / 151 68.1 14.7 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 65 / 66 154.8 6.5 1.0X +InSet expression 162 / 166 61.6 16.2 0.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 117 / 119 85.1 11.7 1.0X +InSet expression 170 / 175 58.8 17.0 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 207 / 208 48.3 20.7 1.0X +InSet expression 211 / 214 47.4 21.1 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 387 / 389 25.9 38.7 1.0X +InSet expression 185 / 187 54.2 18.5 2.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 742 / 744 13.5 74.2 1.0X +InSet expression 172 / 173 58.3 17.2 4.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 floats: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 88 / 91 113.0 8.8 1.0X +InSet expression 170 / 171 58.9 17.0 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 floats: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 129 / 132 77.5 12.9 1.0X +InSet expression 188 / 189 53.2 18.8 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 floats: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 243 / 244 41.2 24.3 1.0X +InSet expression 192 / 194 52.2 19.2 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 floats: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 421 / 424 23.7 42.1 1.0X +InSet expression 237 / 240 42.2 23.7 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 floats: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 775 / 777 12.9 77.5 1.0X +InSet expression 205 / 209 48.8 20.5 3.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 floats: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 3052 / 3151 3.3 305.2 1.0X +InSet expression 197 / 199 50.8 19.7 15.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 doubles: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 82 / 84 121.6 8.2 1.0X +InSet expression 167 / 169 60.0 16.7 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 doubles: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 124 / 131 80.3 12.4 1.0X +InSet expression 186 / 187 53.9 18.6 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 doubles: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 237 / 239 42.1 23.7 1.0X +InSet expression 193 / 194 51.8 19.3 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 doubles: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 416 / 418 24.0 41.6 1.0X +InSet expression 239 / 241 41.8 23.9 1.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 doubles: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 771 / 774 13.0 77.1 1.0X +InSet expression 204 / 207 49.1 20.4 3.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 doubles: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 3755 / 3801 2.7 375.5 1.0X +InSet expression 194 / 197 51.5 19.4 19.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 small decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 46 / 49 21.6 46.4 1.0X +InSet expression 136 / 141 7.4 135.7 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 small decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 57 / 61 17.5 57.1 1.0X +InSet expression 137 / 140 7.3 137.2 0.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 small decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 89 / 92 11.2 89.4 1.0X +InSet expression 139 / 141 7.2 138.7 0.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 small decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 170 / 171 5.9 169.5 1.0X +InSet expression 146 / 148 6.9 145.8 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 small decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 429 / 431 2.3 429.2 1.0X +InSet expression 145 / 148 6.9 144.9 3.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 small decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 995 / 1207 1.0 995.0 1.0X +InSet expression 154 / 158 6.5 154.1 6.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 large decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 268 / 307 3.7 268.3 1.0X +InSet expression 171 / 176 5.8 171.1 1.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 large decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 455 / 458 2.2 455.2 1.0X +InSet expression 173 / 176 5.8 173.1 2.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 large decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 1095 / 1099 0.9 1095.2 1.0X +InSet expression 179 / 183 5.6 178.7 6.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 large decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 2099 / 2110 0.5 2098.6 1.0X +InSet expression 183 / 187 5.5 183.2 11.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 large decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 3885 / 3911 0.3 3885.4 1.0X +InSet expression 207 / 223 4.8 206.6 18.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 large decimals: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 7759 / 7867 0.1 7759.2 1.0X +InSet expression 214 / 217 4.7 214.4 36.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 strings: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 126 / 127 7.9 126.0 1.0X +InSet expression 139 / 142 7.2 139.0 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 strings: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 128 / 132 7.8 128.2 1.0X +InSet expression 142 / 144 7.0 142.0 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 strings: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 151 / 153 6.6 150.9 1.0X +InSet expression 150 / 152 6.7 150.1 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 strings: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 238 / 240 4.2 238.5 1.0X +InSet expression 152 / 154 6.6 152.4 1.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 strings: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 431 / 432 2.3 431.2 1.0X +InSet expression 149 / 151 6.7 148.8 2.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 strings: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 822 / 1060 1.2 821.7 1.0X +InSet expression 153 / 162 6.5 152.9 5.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 timestamps: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 42 / 44 240.5 4.2 1.0X +InSet expression 158 / 161 63.5 15.8 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 timestamps: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 57 / 59 174.5 5.7 1.0X +InSet expression 173 / 176 57.8 17.3 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 timestamps: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 110 / 113 91.1 11.0 1.0X +InSet expression 223 / 226 44.9 22.3 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 timestamps: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 190 / 193 52.6 19.0 1.0X +InSet expression 238 / 240 42.1 23.8 0.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 timestamps: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 355 / 367 28.2 35.5 1.0X +InSet expression 221 / 222 45.2 22.1 1.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 timestamps: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 692 / 694 14.5 69.2 1.0X +InSet expression 220 / 222 45.4 22.0 3.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 dates: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 143 / 145 70.0 14.3 1.0X +InSet expression 264 / 269 37.9 26.4 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 dates: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 141 / 142 71.1 14.1 1.0X +InSet expression 268 / 269 37.3 26.8 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 dates: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 196 / 196 51.1 19.6 1.0X +InSet expression 277 / 282 36.1 27.7 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 dates: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 338 / 351 29.5 33.8 1.0X +InSet expression 287 / 290 34.9 28.7 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 dates: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 541 / 542 18.5 54.1 1.0X +InSet expression 299 / 300 33.5 29.9 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 dates: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 883 / 900 11.3 88.3 1.0X +InSet expression 296 / 298 33.8 29.6 3.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 arrays: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 51 / 53 19.6 51.0 1.0X +InSet expression 96 / 97 10.5 95.7 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 arrays: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 77 / 79 13.1 76.6 1.0X +InSet expression 96 / 98 10.4 96.0 0.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 arrays: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 275 / 276 3.6 274.6 1.0X +InSet expression 119 / 121 8.4 119.1 2.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 arrays: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 592 / 663 1.7 592.1 1.0X +InSet expression 164 / 172 6.1 164.3 3.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 arrays: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 2555 / 2733 0.4 2554.7 1.0X +InSet expression 194 / 198 5.2 193.9 13.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 arrays: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 9215 / 9778 0.1 9214.8 1.0X +InSet expression 253 / 256 3.9 253.2 36.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 structs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 46 / 47 22.0 45.5 1.0X +InSet expression 157 / 162 6.4 156.5 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 structs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 61 / 63 16.5 60.7 1.0X +InSet expression 158 / 161 6.3 158.2 0.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 structs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 133 / 140 7.5 132.8 1.0X +InSet expression 199 / 202 5.0 198.8 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 structs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 369 / 372 2.7 369.1 1.0X +InSet expression 283 / 294 3.5 282.7 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 structs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 1570 / 1731 0.6 1569.8 1.0X +InSet expression 332 / 334 3.0 332.0 4.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 structs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +In expression 6332 / 6794 0.2 6331.8 1.0X +InSet expression 441 / 444 2.3 440.9 14.4X + + diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java index 7e76a651ba2c..117e98f33a0e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java @@ -226,10 +226,10 @@ public void free() { } /** - * Gets the average hash map probe per looking up for the underlying `BytesToBytesMap`. + * Gets the average bucket list iterations per lookup in the underlying `BytesToBytesMap`. */ - public double getAverageProbesPerLookup() { - return map.getAverageProbesPerLookup(); + public double getAvgHashProbeBucketListIterations() { + return map.getAvgHashProbeBucketListIterations(); } /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 9eb03430a7db..5b126338f33b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -25,6 +25,7 @@ import org.apache.spark.SparkEnv; import org.apache.spark.TaskContext; +import org.apache.spark.internal.config.package$; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.SerializerManager; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; @@ -93,8 +94,7 @@ public UnsafeKVExternalSorter( taskContext, comparatorSupplier, prefixComparator, - SparkEnv.get().conf().getInt("spark.shuffle.sort.initialBufferSize", - UnsafeExternalRowSorter.DEFAULT_INITIAL_SORT_BUFFER_SIZE), + (int) SparkEnv.get().conf().get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()), pageSizeBytes, numElementsForSpillThreshold, canUseRadixSort); @@ -160,8 +160,7 @@ public UnsafeKVExternalSorter( taskContext, comparatorSupplier, prefixComparator, - SparkEnv.get().conf().getInt("spark.shuffle.sort.initialBufferSize", - UnsafeExternalRowSorter.DEFAULT_INITIAL_SORT_BUFFER_SIZE), + (int) SparkEnv.get().conf().get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()), pageSizeBytes, numElementsForSpillThreshold, inMemSorter); diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java deleted file mode 100644 index df439e2c02fe..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java +++ /dev/null @@ -1,59 +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.sql.sources.v2; - -import java.util.Optional; - -import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport; -import org.apache.spark.sql.types.StructType; - -/** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data writing ability for batch processing. - * - * This interface is used to create {@link BatchWriteSupport} instances when end users run - * {@code Dataset.write.format(...).option(...).save()}. - */ -@Evolving -public interface BatchWriteSupportProvider extends DataSourceV2 { - - /** - * Creates an optional {@link BatchWriteSupport} instance to save the data to this data source, - * which is called by Spark at the beginning of each batch query. - * - * Data sources can return None if there is no writing needed to be done according to the save - * mode. - * - * @param queryId A unique string for the writing query. It's possible that there are many - * writing queries running at the same time, and the returned - * {@link BatchWriteSupport} can use this id to distinguish itself from others. - * @param schema the schema of the data to be written. - * @param mode the save mode which determines what to do when the data are already in this data - * source, please refer to {@link SaveMode} for more details. - * @param options the options for the returned data source writer, which is an immutable - * case-insensitive string-to-string map. - * @return a write support to write data to this data source. - */ - Optional createBatchWriteSupport( - String queryId, - StructType schema, - SaveMode mode, - DataSourceOptions options); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java index 1c5e3a0cd31e..00af0bf1b172 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java @@ -163,6 +163,11 @@ public double getDouble(String key, double defaultValue) { */ public static final String DATABASE_KEY = "database"; + /** + * The option key for whether to check existence of files for a table. + */ + public static final String CHECK_FILES_EXIST_KEY = "check_files_exist"; + /** * Returns all the paths specified by both the singular path option and the multiple * paths option. @@ -197,4 +202,9 @@ public Optional tableName() { public Optional databaseName() { return get(DATABASE_KEY); } + + public Boolean checkFilesExist() { + Optional result = get(CHECK_FILES_EXIST_KEY); + return result.isPresent() && result.get().equals("true"); + } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java index eae7a45d1d44..4aaa57dd4db9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java @@ -20,15 +20,7 @@ import org.apache.spark.annotation.Evolving; /** - * The base interface for data source v2. Implementations must have a public, 0-arg constructor. - * - * Note that this is an empty interface. Data source implementations must mix in interfaces such as - * {@link BatchReadSupportProvider} or {@link BatchWriteSupportProvider}, which can provide - * batch or streaming read/write support instances. Otherwise it's just a dummy data source which - * is un-readable/writable. - * - * If Spark fails to execute any methods in the implementations of this interface (by throwing an - * exception), the read action will fail and no Spark job will be submitted. + * TODO: remove it when we finish the API refactor for streaming side. */ @Evolving public interface DataSourceV2 {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchWrite.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchWrite.java new file mode 100644 index 000000000000..08caadd5308e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsBatchWrite.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.writer.WriteBuilder; + +/** + * An empty mix-in interface for {@link Table}, to indicate this table supports batch write. + *

+ * If a {@link Table} implements this interface, the + * {@link SupportsWrite#newWriteBuilder(DataSourceOptions)} must return a {@link WriteBuilder} + * with {@link WriteBuilder#buildForBatch()} implemented. + *

+ */ +@Evolving +public interface SupportsBatchWrite extends SupportsWrite {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java index e22738d20d50..5031c71c0fd4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java @@ -29,7 +29,10 @@ interface SupportsRead extends Table { /** * Returns a {@link ScanBuilder} which can be used to build a {@link Scan}. Spark will call this - * method to configure each scan. + * method to configure each data source scan. + * + * @param options The options for reading, which is an immutable case-insensitive + * string-to-string map. */ ScanBuilder newScanBuilder(DataSourceOptions options); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java new file mode 100644 index 000000000000..ecdfe2073025 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2; + +import org.apache.spark.sql.sources.v2.writer.BatchWrite; +import org.apache.spark.sql.sources.v2.writer.WriteBuilder; + +/** + * An internal base interface of mix-in interfaces for writable {@link Table}. This adds + * {@link #newWriteBuilder(DataSourceOptions)} that is used to create a write + * for batch or streaming. + */ +interface SupportsWrite extends Table { + + /** + * Returns a {@link WriteBuilder} which can be used to create {@link BatchWrite}. Spark will call + * this method to configure each data source write. + */ + WriteBuilder newWriteBuilder(DataSourceOptions options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java similarity index 99% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java rename to sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java index efe1ac4f78db..91297759971b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java @@ -38,7 +38,7 @@ * Please refer to the documentation of commit/abort methods for detailed specifications. */ @Evolving -public interface BatchWriteSupport { +public interface BatchWrite { /** * Creates a writer factory which will be serialized and sent to executors. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java index d142ee523ef9..11228ad1ea67 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java @@ -36,11 +36,11 @@ * * If this data writer succeeds(all records are successfully written and {@link #commit()} * succeeds), a {@link WriterCommitMessage} will be sent to the driver side and pass to - * {@link BatchWriteSupport#commit(WriterCommitMessage[])} with commit messages from other data + * {@link BatchWrite#commit(WriterCommitMessage[])} with commit messages from other data * writers. If this data writer fails(one record fails to write or {@link #commit()} fails), an * exception will be sent to the driver side, and Spark may retry this writing task a few times. * In each retry, {@link DataWriterFactory#createWriter(int, long)} will receive a - * different `taskId`. Spark will call {@link BatchWriteSupport#abort(WriterCommitMessage[])} + * different `taskId`. Spark will call {@link BatchWrite#abort(WriterCommitMessage[])} * when the configured number of retries is exhausted. * * Besides the retry mechanism, Spark may launch speculative tasks if the existing writing task @@ -71,11 +71,11 @@ public interface DataWriter { /** * Commits this writer after all records are written successfully, returns a commit message which * will be sent back to driver side and passed to - * {@link BatchWriteSupport#commit(WriterCommitMessage[])}. + * {@link BatchWrite#commit(WriterCommitMessage[])}. * * The written data should only be visible to data source readers after - * {@link BatchWriteSupport#commit(WriterCommitMessage[])} succeeds, which means this method - * should still "hide" the written data and ask the {@link BatchWriteSupport} at driver side to + * {@link BatchWrite#commit(WriterCommitMessage[])} succeeds, which means this method + * should still "hide" the written data and ask the {@link BatchWrite} at driver side to * do the final commit via {@link WriterCommitMessage}. * * If this method fails (by throwing an exception), {@link #abort()} will be called and this @@ -93,7 +93,7 @@ public interface DataWriter { * failed. * * If this method fails(by throwing an exception), the underlying data source may have garbage - * that need to be cleaned by {@link BatchWriteSupport#abort(WriterCommitMessage[])} or manually, + * that need to be cleaned by {@link BatchWrite#abort(WriterCommitMessage[])} or manually, * but these garbage should not be visible to data source readers. * * @throws IOException if failure happens during disk/network IO like writing files. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java index 65105f46b82d..bf2db9059b08 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow; /** - * A factory of {@link DataWriter} returned by {@link BatchWriteSupport#createBatchWriterFactory()}, + * A factory of {@link DataWriter} returned by {@link BatchWrite#createBatchWriterFactory()}, * which is responsible for creating and initializing the actual data writer at executor side. * * Note that, the writer factory will be serialized and sent to executors, then the data writer diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsSaveMode.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsSaveMode.java new file mode 100644 index 000000000000..c4295f237187 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsSaveMode.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.writer; + +import org.apache.spark.sql.SaveMode; + +// A temporary mixin trait for `WriteBuilder` to support `SaveMode`. Will be removed before +// Spark 3.0 when all the new write operators are finished. See SPARK-26356 for more details. +public interface SupportsSaveMode extends WriteBuilder { + WriteBuilder mode(SaveMode mode); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java new file mode 100644 index 000000000000..e861c72af9e6 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.writer; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.SupportsBatchWrite; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.types.StructType; + +/** + * An interface for building the {@link BatchWrite}. Implementations can mix in some interfaces to + * support different ways to write data to data sources. + * + * Unless modified by a mixin interface, the {@link BatchWrite} configured by this builder is to + * append data without affecting existing data. + */ +@Evolving +public interface WriteBuilder { + + /** + * Passes the `queryId` from Spark to data source. `queryId` is a unique string of the query. It's + * possible that there are many queries running at the same time, or a query is restarted and + * resumed. {@link BatchWrite} can use this id to identify the query. + * + * @return a new builder with the `queryId`. By default it returns `this`, which means the given + * `queryId` is ignored. Please override this method to take the `queryId`. + */ + default WriteBuilder withQueryId(String queryId) { + return this; + } + + /** + * Passes the schema of the input data from Spark to data source. + * + * @return a new builder with the `schema`. By default it returns `this`, which means the given + * `schema` is ignored. Please override this method to take the `schema`. + */ + default WriteBuilder withInputDataSchema(StructType schema) { + return this; + } + + /** + * Returns a {@link BatchWrite} to write data to batch source. By default this method throws + * exception, data sources must overwrite this method to provide an implementation, if the + * {@link Table} that creates this scan implements {@link SupportsBatchWrite}. + * + * Note that, the returned {@link BatchWrite} can be null if the implementation supports SaveMode, + * to indicate that no writing is needed. We can clean it up after removing + * {@link SupportsSaveMode}. + */ + default BatchWrite buildForBatch() { + throw new UnsupportedOperationException("Batch scans are not supported"); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java index 9216e3439909..6334c8f64309 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java @@ -24,7 +24,7 @@ /** * A commit message returned by {@link DataWriter#commit()} and will be sent back to the driver side - * as the input parameter of {@link BatchWriteSupport#commit(WriterCommitMessage[])} or + * as the input parameter of {@link BatchWrite#commit(WriterCommitMessage[])} or * {@link StreamingWriteSupport#commit(long, WriterCommitMessage[])}. * * This is an empty interface, data sources should define their own message class and use it when diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 1b37905543b4..7cdfddc5e7aa 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1,6 +1,7 @@ org.apache.spark.sql.execution.datasources.csv.CSVFileFormat org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider org.apache.spark.sql.execution.datasources.json.JsonFileFormat +org.apache.spark.sql.execution.datasources.noop.NoopDataSource org.apache.spark.sql.execution.datasources.orc.OrcFileFormat org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat org.apache.spark.sql.execution.datasources.text.TextFileFormat diff --git a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js index 5161fcde669e..46d3fbc8c3cb 100644 --- a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js +++ b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js @@ -42,7 +42,7 @@ function renderPlanViz() { setupTooltipForSparkPlanNode(i); } - resizeSvg(svg) + resizeSvg(svg); } /* -------------------- * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index ce8e4c8f5b82..2b1521730bc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -37,10 +37,9 @@ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.csv._ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils, FileDataSourceV2, FileTable} import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String /** @@ -193,7 +192,16 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { "read files of Hive data source directly.") } - val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) + val useV1Sources = + sparkSession.sessionState.conf.userV1SourceReaderList.toLowerCase(Locale.ROOT).split(",") + val lookupCls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) + val cls = lookupCls.newInstance() match { + case f: FileDataSourceV2 if useV1Sources.contains(f.shortName()) || + useV1Sources.contains(lookupCls.getCanonicalName.toLowerCase(Locale.ROOT)) => + f.fallBackFileFormat + case _ => lookupCls + } + if (classOf[TableProvider].isAssignableFrom(cls)) { val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] val sessionOptions = DataSourceV2Utils.extractSessionConfigs( @@ -202,16 +210,16 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val objectMapper = new ObjectMapper() DataSourceOptions.PATHS_KEY -> objectMapper.writeValueAsString(paths.toArray) } - val finalOptions = sessionOptions ++ extraOptions.toMap + pathsOption + val checkFilesExistsOption = DataSourceOptions.CHECK_FILES_EXIST_KEY -> "true" + val finalOptions = sessionOptions ++ extraOptions.toMap + pathsOption + checkFilesExistsOption val dsOptions = new DataSourceOptions(finalOptions.asJava) val table = userSpecifiedSchema match { case Some(schema) => provider.getTable(dsOptions, schema) case _ => provider.getTable(dsOptions) } table match { - case s: SupportsBatchRead => - Dataset.ofRows(sparkSession, DataSourceV2Relation.create( - provider, s, finalOptions, userSpecifiedSchema = userSpecifiedSchema)) + case _: SupportsBatchRead => + Dataset.ofRows(sparkSession, DataSourceV2Relation.create(table, finalOptions)) case _ => loadV1Source(paths: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 981b3a8fd4ac..d9404cd92992 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, Logi import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils, WriteToDataSourceV2} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.writer.SupportsSaveMode import org.apache.spark.sql.types.StructType /** @@ -241,33 +242,39 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { assertNotBucketed("save") - val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) - if (classOf[DataSourceV2].isAssignableFrom(cls)) { - val source = cls.getConstructor().newInstance().asInstanceOf[DataSourceV2] - source match { - case provider: BatchWriteSupportProvider => - val sessionOptions = DataSourceV2Utils.extractSessionConfigs( - source, - df.sparkSession.sessionState.conf) - val options = sessionOptions ++ extraOptions - + val session = df.sparkSession + val cls = DataSource.lookupDataSource(source, session.sessionState.conf) + if (classOf[TableProvider].isAssignableFrom(cls)) { + val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + provider, session.sessionState.conf) + val checkFilesExistsOption = DataSourceOptions.CHECK_FILES_EXIST_KEY -> "false" + val options = sessionOptions ++ extraOptions + checkFilesExistsOption + val dsOptions = new DataSourceOptions(options.asJava) + provider.getTable(dsOptions) match { + case table: SupportsBatchWrite => if (mode == SaveMode.Append) { - val relation = DataSourceV2Relation.createRelationForWrite(source, options) + val relation = DataSourceV2Relation.create(table, options) runCommand(df.sparkSession, "save") { AppendData.byName(relation, df.logicalPlan) } - } else { - val writer = provider.createBatchWriteSupport( - UUID.randomUUID().toString, - df.logicalPlan.output.toStructType, - mode, - new DataSourceOptions(options.asJava)) - - if (writer.isPresent) { - runCommand(df.sparkSession, "save") { - WriteToDataSourceV2(writer.get, df.logicalPlan) - } + val writeBuilder = table.newWriteBuilder(dsOptions) + .withQueryId(UUID.randomUUID().toString) + .withInputDataSchema(df.logicalPlan.schema) + writeBuilder match { + case s: SupportsSaveMode => + val write = s.mode(mode).buildForBatch() + // It can only return null with `SupportsSaveMode`. We can clean it up after + // removing `SupportsSaveMode`. + if (write != null) { + runCommand(df.sparkSession, "save") { + WriteToDataSourceV2(write, df.logicalPlan) + } + } + + case _ => throw new AnalysisException( + s"data source ${table.name} does not support SaveMode $mode") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 8b84eda36103..f852a5296493 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -319,8 +319,8 @@ case class FileSourceScanExec( override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of files"), - "metadataTime" -> SQLMetrics.createMetric(sparkContext, "metadata time"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of files read"), + "metadataTime" -> SQLMetrics.createTimingMetric(sparkContext, "metadata time"), "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) protected override def doExecute(): RDD[InternalRow] = { @@ -373,8 +373,7 @@ case class FileSourceScanExec( val filesGroupedToBuckets = selectedPartitions.flatMap { p => p.files.filter(_.getLen > 0).map { f => - val hosts = getBlockHosts(getBlockLocations(f), 0, f.getLen) - PartitionedFile(p.values, f.getPath.toUri.toString, 0, f.getLen, hosts) + PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values) } }.groupBy { f => BucketingUtils @@ -410,107 +409,35 @@ case class FileSourceScanExec( readFile: (PartitionedFile) => Iterator[InternalRow], selectedPartitions: Seq[PartitionDirectory], fsRelation: HadoopFsRelation): RDD[InternalRow] = { - val defaultMaxSplitBytes = - fsRelation.sparkSession.sessionState.conf.filesMaxPartitionBytes val openCostInBytes = fsRelation.sparkSession.sessionState.conf.filesOpenCostInBytes - val defaultParallelism = fsRelation.sparkSession.sparkContext.defaultParallelism - val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum - val bytesPerCore = totalBytes / defaultParallelism - - val maxSplitBytes = Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) + val maxSplitBytes = + FilePartition.maxSplitBytes(fsRelation.sparkSession, selectedPartitions) logInfo(s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " + s"open cost is considered as scanning $openCostInBytes bytes.") val splitFiles = selectedPartitions.flatMap { partition => partition.files.filter(_.getLen > 0).flatMap { file => - val blockLocations = getBlockLocations(file) - if (fsRelation.fileFormat.isSplitable( - fsRelation.sparkSession, fsRelation.options, file.getPath)) { - (0L until file.getLen by maxSplitBytes).map { offset => - val remaining = file.getLen - offset - val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining - val hosts = getBlockHosts(blockLocations, offset, size) - PartitionedFile( - partition.values, file.getPath.toUri.toString, offset, size, hosts) - } - } else { - val hosts = getBlockHosts(blockLocations, 0, file.getLen) - Seq(PartitionedFile( - partition.values, file.getPath.toUri.toString, 0, file.getLen, hosts)) - } + // getPath() is very expensive so we only want to call it once in this block: + val filePath = file.getPath + val isSplitable = relation.fileFormat.isSplitable( + relation.sparkSession, relation.options, filePath) + PartitionedFileUtil.splitFiles( + sparkSession = relation.sparkSession, + file = file, + filePath = filePath, + isSplitable = isSplitable, + maxSplitBytes = maxSplitBytes, + partitionValues = partition.values + ) } }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) - val partitions = new ArrayBuffer[FilePartition] - val currentFiles = new ArrayBuffer[PartitionedFile] - var currentSize = 0L - - /** Close the current partition and move to the next. */ - def closePartition(): Unit = { - if (currentFiles.nonEmpty) { - val newPartition = - FilePartition( - partitions.size, - currentFiles.toArray.toSeq) // Copy to a new Array. - partitions += newPartition - } - currentFiles.clear() - currentSize = 0 - } - - // Assign files to partitions using "Next Fit Decreasing" - splitFiles.foreach { file => - if (currentSize + file.length > maxSplitBytes) { - closePartition() - } - // Add the given file to the current partition. - currentSize += file.length + openCostInBytes - currentFiles += file - } - closePartition() + val partitions = + FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) new FileScanRDD(fsRelation.sparkSession, readFile, partitions) } - private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { - case f: LocatedFileStatus => f.getBlockLocations - case f => Array.empty[BlockLocation] - } - - // Given locations of all blocks of a single file, `blockLocations`, and an `(offset, length)` - // pair that represents a segment of the same file, find out the block that contains the largest - // fraction the segment, and returns location hosts of that block. If no such block can be found, - // returns an empty array. - private def getBlockHosts( - blockLocations: Array[BlockLocation], offset: Long, length: Long): Array[String] = { - val candidates = blockLocations.map { - // The fragment starts from a position within this block - case b if b.getOffset <= offset && offset < b.getOffset + b.getLength => - b.getHosts -> (b.getOffset + b.getLength - offset).min(length) - - // The fragment ends at a position within this block - case b if offset <= b.getOffset && offset + length < b.getLength => - b.getHosts -> (offset + length - b.getOffset).min(length) - - // The fragment fully contains this block - case b if offset <= b.getOffset && b.getOffset + b.getLength <= offset + length => - b.getHosts -> b.getLength - - // The fragment doesn't intersect with this block - case b => - b.getHosts -> 0L - }.filter { case (hosts, size) => - size > 0L - } - - if (candidates.isEmpty) { - Array.empty[String] - } else { - val (hosts, _) = candidates.maxBy { case (_, size) => size } - hosts - } - } - override def doCanonicalize(): FileSourceScanExec = { FileSourceScanExec( relation, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala index 5b4edf5136e3..85f49140a4b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala @@ -145,11 +145,12 @@ case class ExpandExec( // Part 1: declare variables for each column // If a column has the same value for all output rows, then we also generate its computation // right after declaration. Otherwise its value is computed in the part 2. + lazy val attributeSeq: AttributeSeq = child.output val outputColumns = output.indices.map { col => val firstExpr = projections.head(col) if (sameOutput(col)) { // This column is the same across all output rows. Just generate code for it here. - BindReferences.bindReference(firstExpr, child.output).genCode(ctx) + BindReferences.bindReference(firstExpr, attributeSeq).genCode(ctx) } else { val isNull = ctx.freshName("isNull") val value = ctx.freshName("value") @@ -170,7 +171,7 @@ case class ExpandExec( var updateCode = "" for (col <- exprs.indices) { if (!sameOutput(col)) { - val ev = BindReferences.bindReference(exprs(col), child.output).genCode(ctx) + val ev = BindReferences.bindReference(exprs(col), attributeSeq).genCode(ctx) updateCode += s""" |${ev.code} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index c90b254a6d12..d3934a0e52de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -21,7 +21,7 @@ import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -77,6 +77,10 @@ object HiveResult { TimestampType, BinaryType) + private lazy val dateFormatter = DateFormatter() + private lazy val timestampFormatter = TimestampFormatter( + DateTimeUtils.getTimeZone(SQLConf.get.sessionLocalTimeZone)) + /** Hive outputs fields of structs slightly differently than top level attributes. */ private def toHiveStructString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => @@ -111,11 +115,9 @@ object HiveResult { toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" - case (d: Date, DateType) => - DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d)) + case (d: Date, DateType) => dateFormatter.format(DateTimeUtils.fromJavaDate(d)) case (t: Timestamp, TimestampType) => - val timeZone = DateTimeUtils.getTimeZone(SQLConf.get.sessionLocalTimeZone) - DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t), timeZone) + DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) case (interval, CalendarIntervalType) => interval.toString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala new file mode 100644 index 000000000000..3196624f7c7c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources._ + +object PartitionedFileUtil { + def splitFiles( + sparkSession: SparkSession, + file: FileStatus, + filePath: Path, + isSplitable: Boolean, + maxSplitBytes: Long, + partitionValues: InternalRow): Seq[PartitionedFile] = { + if (isSplitable) { + (0L until file.getLen by maxSplitBytes).map { offset => + val remaining = file.getLen - offset + val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining + val hosts = getBlockHosts(getBlockLocations(file), offset, size) + PartitionedFile(partitionValues, filePath.toUri.toString, offset, size, hosts) + } + } else { + Seq(getPartitionedFile(file, filePath, partitionValues)) + } + } + + def getPartitionedFile( + file: FileStatus, + filePath: Path, + partitionValues: InternalRow): PartitionedFile = { + val hosts = getBlockHosts(getBlockLocations(file), 0, file.getLen) + PartitionedFile(partitionValues, filePath.toUri.toString, 0, file.getLen, hosts) + } + + private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { + case f: LocatedFileStatus => f.getBlockLocations + case f => Array.empty[BlockLocation] + } + // Given locations of all blocks of a single file, `blockLocations`, and an `(offset, length)` + // pair that represents a segment of the same file, find out the block that contains the largest + // fraction the segment, and returns location hosts of that block. If no such block can be found, + // returns an empty array. + private def getBlockHosts( + blockLocations: Array[BlockLocation], + offset: Long, + length: Long): Array[String] = { + val candidates = blockLocations.map { + // The fragment starts from a position within this block + case b if b.getOffset <= offset && offset < b.getOffset + b.getLength => + b.getHosts -> (b.getOffset + b.getLength - offset).min(length) + + // The fragment ends at a position within this block + case b if offset <= b.getOffset && offset + length < b.getLength => + b.getHosts -> (offset + length - b.getOffset).min(length) + + // The fragment fully contains this block + case b if offset <= b.getOffset && b.getOffset + b.getLength <= offset + length => + b.getHosts -> b.getLength + + // The fragment doesn't intersect with this block + case b => + b.getHosts -> 0L + }.filter { case (hosts, size) => + size > 0L + } + + if (candidates.isEmpty) { + Array.empty[String] + } else { + val (hosts, _) = candidates.maxBy { case (_, size) => size } + hosts + } + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala index 98c4a5129995..a1fb23d621d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala @@ -77,6 +77,7 @@ abstract class AggregationIterator( val expressionsLength = expressions.length val functions = new Array[AggregateFunction](expressionsLength) var i = 0 + val inputAttributeSeq: AttributeSeq = inputAttributes while (i < expressionsLength) { val func = expressions(i).aggregateFunction val funcWithBoundReferences: AggregateFunction = expressions(i).mode match { @@ -86,7 +87,7 @@ abstract class AggregationIterator( // this function is Partial or Complete because we will call eval of this // function's children in the update method of this aggregate function. // Those eval calls require BoundReferences to work. - BindReferences.bindReference(func, inputAttributes) + BindReferences.bindReference(func, inputAttributeSeq) case _ => // We only need to set inputBufferOffset for aggregate functions with mode // PartialMerge and Final. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 2355d305c38e..19a47ffc6dd0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -23,6 +23,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -62,8 +63,9 @@ case class HashAggregateExec( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"), "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"), - "aggTime" -> SQLMetrics.createTimingMetric(sparkContext, "aggregate time"), - "avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe")) + "aggTime" -> SQLMetrics.createTimingMetric(sparkContext, "time in aggregation build"), + "avgHashProbe" -> + SQLMetrics.createAverageMetric(sparkContext, "avg hash probe bucket list iters")) override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) @@ -199,15 +201,13 @@ case class HashAggregateExec( val (resultVars, genResult) = if (modes.contains(Final) || modes.contains(Complete)) { // evaluate aggregate results ctx.currentVars = bufVars - val aggResults = functions.map(_.evaluateExpression).map { e => - BindReferences.bindReference(e, aggregateBufferAttributes).genCode(ctx) - } + val aggResults = bindReferences( + functions.map(_.evaluateExpression), + aggregateBufferAttributes).map(_.genCode(ctx)) val evaluateAggResults = evaluateVariables(aggResults) // evaluate result expressions ctx.currentVars = aggResults - val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, aggregateAttributes).genCode(ctx) - } + val resultVars = bindReferences(resultExpressions, aggregateAttributes).map(_.genCode(ctx)) (resultVars, s""" |$evaluateAggResults |${evaluateVariables(resultVars)} @@ -264,7 +264,7 @@ case class HashAggregateExec( } } ctx.currentVars = bufVars ++ input - val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttrs)) + val boundUpdateExpr = bindReferences(updateExpr, inputAttrs) val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) val effectiveCodes = subExprs.codes.mkString("\n") val aggVals = ctx.withSubExprEliminationExprs(subExprs.states) { @@ -363,7 +363,7 @@ case class HashAggregateExec( metrics.incPeakExecutionMemory(maxMemory) // Update average hashmap probe - avgHashProbe.set(hashMap.getAverageProbesPerLookup()) + avgHashProbe.set(hashMap.getAvgHashProbeBucketListIterations) if (sorter == null) { // not spilled @@ -456,16 +456,16 @@ case class HashAggregateExec( val evaluateBufferVars = evaluateVariables(bufferVars) // evaluate the aggregation result ctx.currentVars = bufferVars - val aggResults = declFunctions.map(_.evaluateExpression).map { e => - BindReferences.bindReference(e, aggregateBufferAttributes).genCode(ctx) - } + val aggResults = bindReferences( + declFunctions.map(_.evaluateExpression), + aggregateBufferAttributes).map(_.genCode(ctx)) val evaluateAggResults = evaluateVariables(aggResults) // generate the final result ctx.currentVars = keyVars ++ aggResults val inputAttrs = groupingAttributes ++ aggregateAttributes - val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, inputAttrs).genCode(ctx) - } + val resultVars = bindReferences[Expression]( + resultExpressions, + inputAttrs).map(_.genCode(ctx)) s""" $evaluateKeyVars $evaluateBufferVars @@ -494,9 +494,9 @@ case class HashAggregateExec( ctx.currentVars = keyVars ++ resultBufferVars val inputAttrs = resultExpressions.map(_.toAttribute) - val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, inputAttrs).genCode(ctx) - } + val resultVars = bindReferences[Expression]( + resultExpressions, + inputAttrs).map(_.genCode(ctx)) s""" $evaluateKeyVars $evaluateResultBufferVars @@ -506,9 +506,9 @@ case class HashAggregateExec( // generate result based on grouping key ctx.INPUT_ROW = keyTerm ctx.currentVars = null - val eval = resultExpressions.map{ e => - BindReferences.bindReference(e, groupingAttributes).genCode(ctx) - } + val eval = bindReferences[Expression]( + resultExpressions, + groupingAttributes).map(_.genCode(ctx)) consume(ctx, eval) } ctx.addNewFunction(funcName, @@ -730,9 +730,9 @@ case class HashAggregateExec( private def doConsumeWithKeys(ctx: CodegenContext, input: Seq[ExprCode]): String = { // create grouping key val unsafeRowKeyCode = GenerateUnsafeProjection.createCode( - ctx, groupingExpressions.map(e => BindReferences.bindReference[Expression](e, child.output))) + ctx, bindReferences[Expression](groupingExpressions, child.output)) val fastRowKeys = ctx.generateExpressions( - groupingExpressions.map(e => BindReferences.bindReference[Expression](e, child.output))) + bindReferences[Expression](groupingExpressions, child.output)) val unsafeRowKeys = unsafeRowKeyCode.value val unsafeRowBuffer = ctx.freshName("unsafeRowAggBuffer") val fastRowBuffer = ctx.freshName("fastAggBuffer") @@ -825,7 +825,7 @@ case class HashAggregateExec( val updateRowInRegularHashMap: String = { ctx.INPUT_ROW = unsafeRowBuffer - val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) + val boundUpdateExpr = bindReferences(updateExpr, inputAttr) val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) val effectiveCodes = subExprs.codes.mkString("\n") val unsafeRowBufferEvals = ctx.withSubExprEliminationExprs(subExprs.states) { @@ -849,7 +849,7 @@ case class HashAggregateExec( if (isFastHashMapEnabled) { if (isVectorizedHashMapEnabled) { ctx.INPUT_ROW = fastRowBuffer - val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) + val boundUpdateExpr = bindReferences(updateExpr, inputAttr) val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) val effectiveCodes = subExprs.codes.mkString("\n") val fastRowEvals = ctx.withSubExprEliminationExprs(subExprs.states) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index bd52c6321647..5b340eead39e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -77,7 +77,7 @@ case class ObjectHashAggregateExec( override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "aggTime" -> SQLMetrics.createTimingMetric(sparkContext, "aggregate time") + "aggTime" -> SQLMetrics.createTimingMetric(sparkContext, "time in aggregation build") ) override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 6d849869b577..6dc64657ebf1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -387,7 +387,7 @@ class TungstenAggregationIterator( metrics.incPeakExecutionMemory(maxMemory) // Updating average hashmap probe - avgHashProbe.set(hashMap.getAverageProbesPerLookup()) + avgHashProbe.set(hashMap.getAvgHashProbeBucketListIterations) }) /////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 2570b36b3166..4352721b46b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -24,6 +24,7 @@ import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskCon import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics @@ -56,7 +57,7 @@ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) } override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { - val exprs = projectList.map(x => BindReferences.bindReference[Expression](x, child.output)) + val exprs = bindReferences[Expression](projectList, child.output) val resultVars = exprs.map(_.genCode(ctx)) // Evaluation of non-deterministic expressions can't be deferred. val nonDeterministicAttrs = projectList.filterNot(_.deterministic).map(_.toAttribute) @@ -662,8 +663,8 @@ object CoalesceExec { case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode { override lazy val metrics = Map( - "dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"), - "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)")) + "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), + "collectTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to collect")) override def output: Seq[Attribute] = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 754a3316ffb7..a1f2785ed362 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -110,6 +110,9 @@ case class DataWritingCommandExec(cmd: DataWritingCommand, child: SparkPlan) override def nodeName: String = "Execute " + cmd.nodeName + // override the default one, otherwise the `cmd.nodeName` will appear twice from simpleString + override def argString(maxFields: Int): String = cmd.argString(maxFields) + override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray override def executeToIterator: Iterator[InternalRow] = sideEffectResult.toIterator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index e2cd40906f40..d24e66e58385 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -214,7 +215,7 @@ case class AlterTableAddColumnsCommand( /** * ALTER TABLE ADD COLUMNS command does not support temporary view/table, * view, or datasource table with text, orc formats or external provider. - * For datasource table, it currently only supports parquet, json, csv. + * For datasource table, it currently only supports parquet, json, csv, orc. */ private def verifyAlterTableAddColumn( conf: SQLConf, @@ -237,7 +238,7 @@ case class AlterTableAddColumnsCommand( // TextFileFormat only default to one column "value" // Hive type is already considered as hive serde table, so the logic will not // come in here. - case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat => + case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat | _: OrcDataSourceV2 => case s if s.getClass.getCanonicalName.endsWith("OrcFileFormat") => case s => throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala index ba7d2b7cbdb1..b71c2d12f02b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala @@ -173,7 +173,7 @@ object BasicWriteJobStatsTracker { val sparkContext = SparkContext.getActive.get Map( NUM_FILES_KEY -> SQLMetrics.createMetric(sparkContext, "number of written files"), - NUM_OUTPUT_BYTES_KEY -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + NUM_OUTPUT_BYTES_KEY -> SQLMetrics.createSizeMetric(sparkContext, "written output"), NUM_OUTPUT_ROWS_KEY -> SQLMetrics.createMetric(sparkContext, "number of output rows"), NUM_PARTS_KEY -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 5dad784e45af..d48261e783dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import scala.language.{existentials, implicitConversions} import scala.util.{Failure, Success, Try} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.deploy.SparkHadoopUtil @@ -40,6 +41,8 @@ import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf @@ -90,8 +93,19 @@ case class DataSource( case class SourceInfo(name: String, schema: StructType, partitionColumns: Seq[String]) - lazy val providingClass: Class[_] = - DataSource.lookupDataSource(className, sparkSession.sessionState.conf) + lazy val providingClass: Class[_] = { + val cls = DataSource.lookupDataSource(className, sparkSession.sessionState.conf) + // `providingClass` is used for resolving data source relation for catalog tables. + // As now catalog for data source V2 is under development, here we fall back all the + // [[FileDataSourceV2]] to [[FileFormat]] to guarantee the current catalog works. + // [[FileDataSourceV2]] will still be used if we call the load()/save() method in + // [[DataFrameReader]]/[[DataFrameWriter]], since they use method `lookupDataSource` + // instead of `providingClass`. + cls.newInstance() match { + case f: FileDataSourceV2 => f.fallBackFileFormat + case _ => cls + } + } lazy val sourceInfo: SourceInfo = sourceSchema() private val caseInsensitiveOptions = CaseInsensitiveMap(options) private val equality = sparkSession.sessionState.conf.resolver @@ -543,40 +557,9 @@ case class DataSource( checkFilesExist: Boolean): Seq[Path] = { val allPaths = caseInsensitiveOptions.get("path") ++ paths val hadoopConf = sparkSession.sessionState.newHadoopConf() - val allGlobPath = allPaths.flatMap { path => - val hdfsPath = new Path(path) - val fs = hdfsPath.getFileSystem(hadoopConf) - val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - val globPath = SparkHadoopUtil.get.globPathIfNecessary(fs, qualified) - - if (checkEmptyGlobPath && globPath.isEmpty) { - throw new AnalysisException(s"Path does not exist: $qualified") - } - - // Sufficient to check head of the globPath seq for non-glob scenario - // Don't need to check once again if files exist in streaming mode - if (checkFilesExist && !fs.exists(globPath.head)) { - throw new AnalysisException(s"Path does not exist: ${globPath.head}") - } - globPath - }.toSeq - if (checkFilesExist) { - val (filteredOut, filteredIn) = allGlobPath.partition { path => - InMemoryFileIndex.shouldFilterOut(path.getName) - } - if (filteredOut.nonEmpty) { - if (filteredIn.isEmpty) { - logWarning( - s"All paths were ignored:\n ${filteredOut.mkString("\n ")}") - } else { - logDebug( - s"Some paths were ignored:\n ${filteredOut.mkString("\n ")}") - } - } - } - - allGlobPath + DataSource.checkAndGlobPathIfNecessary(allPaths.toSeq, hadoopConf, + checkEmptyGlobPath, checkFilesExist) } } @@ -632,7 +615,7 @@ object DataSource extends Logging { val provider1 = backwardCompatibilityMap.getOrElse(provider, provider) match { case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" => - classOf[OrcFileFormat].getCanonicalName + classOf[OrcDataSourceV2].getCanonicalName case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "hive" => "org.apache.spark.sql.hive.orc.OrcFileFormat" @@ -721,6 +704,48 @@ object DataSource extends Logging { } } + /** + * Checks and returns files in all the paths. + */ + private[sql] def checkAndGlobPathIfNecessary( + paths: Seq[String], + hadoopConf: Configuration, + checkEmptyGlobPath: Boolean, + checkFilesExist: Boolean): Seq[Path] = { + val allGlobPath = paths.flatMap { path => + val hdfsPath = new Path(path) + val fs = hdfsPath.getFileSystem(hadoopConf) + val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + val globPath = SparkHadoopUtil.get.globPathIfNecessary(fs, qualified) + + if (checkEmptyGlobPath && globPath.isEmpty) { + throw new AnalysisException(s"Path does not exist: $qualified") + } + + // Sufficient to check head of the globPath seq for non-glob scenario + // Don't need to check once again if files exist in streaming mode + if (checkFilesExist && !fs.exists(globPath.head)) { + throw new AnalysisException(s"Path does not exist: ${globPath.head}") + } + globPath + } + + if (checkFilesExist) { + val (filteredOut, filteredIn) = allGlobPath.partition { path => + InMemoryFileIndex.shouldFilterOut(path.getName) + } + if (filteredIn.isEmpty) { + logWarning( + s"All paths were ignored:\n ${filteredOut.mkString("\n ")}") + } else { + logDebug( + s"Some paths were ignored:\n ${filteredOut.mkString("\n ")}") + } + } + + allGlobPath + } + /** * When creating a data source table, the `path` option has a special meaning: the table location. * This method extracts the `path` option and treat it as table location to build a diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala new file mode 100644 index 000000000000..cefdfd14b3b3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallbackOrcDataSourceV2.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2} +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable + +/** + * Replace the ORC V2 data source of table in [[InsertIntoTable]] to V1 [[FileFormat]]. + * E.g, with temporary view `t` using [[FileDataSourceV2]], inserting into view `t` fails + * since there is no corresponding physical plan. + * SPARK-23817: This is a temporary hack for making current data source V2 work. It should be + * removed when write path of file data source v2 is finished. + */ +class FallbackOrcDataSourceV2(sparkSession: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case i @ InsertIntoTable(d @DataSourceV2Relation(table: OrcTable, _, _), _, _, _, _) => + val v1FileFormat = new OrcFileFormat + val relation = HadoopFsRelation(table.getFileIndex, table.getFileIndex.partitionSchema, + table.schema(), None, v1FileFormat, d.options)(sparkSession) + i.copy(table = LogicalRelation(relation)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 774fe38f5c2e..260ad97506a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.execution.{SortExec, SparkPlan, SQLExecution} @@ -145,9 +146,8 @@ object FileFormatWriter extends Logging { // SPARK-21165: the `requiredOrdering` is based on the attributes from analyzed plan, and // the physical plan may have different attribute ids due to optimizer removing some // aliases. Here we bind the expression ahead to avoid potential attribute ids mismatch. - val orderingExpr = requiredOrdering - .map(SortOrder(_, Ascending)) - .map(BindReferences.bindReference(_, outputSpec.outputColumns)) + val orderingExpr = bindReferences( + requiredOrdering.map(SortOrder(_, Ascending)), outputSpec.outputColumns) SortExec( orderingExpr, global = false, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala new file mode 100644 index 000000000000..4b1ade8e2957 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.Partition +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.sources.v2.reader.InputPartition + +/** + * A collection of file blocks that should be read as a single task + * (possibly from multiple partitioned directories). + */ +case class FilePartition(index: Int, files: Seq[PartitionedFile]) + extends Partition with InputPartition { + override def preferredLocations(): Array[String] = { + // Computes total number of bytes can be retrieved from each host. + val hostToNumBytes = mutable.HashMap.empty[String, Long] + files.foreach { file => + file.locations.filter(_ != "localhost").foreach { host => + hostToNumBytes(host) = hostToNumBytes.getOrElse(host, 0L) + file.length + } + } + + // Takes the first 3 hosts with the most data to be retrieved + hostToNumBytes.toSeq.sortBy { + case (host, numBytes) => numBytes + }.reverse.take(3).map { + case (host, numBytes) => host + }.toArray + } +} + +object FilePartition extends Logging { + + def getFilePartitions( + sparkSession: SparkSession, + partitionedFiles: Seq[PartitionedFile], + maxSplitBytes: Long): Seq[FilePartition] = { + val partitions = new ArrayBuffer[FilePartition] + val currentFiles = new ArrayBuffer[PartitionedFile] + var currentSize = 0L + + /** Close the current partition and move to the next. */ + def closePartition(): Unit = { + if (currentFiles.nonEmpty) { + // Copy to a new Array. + val newPartition = FilePartition(partitions.size, currentFiles.toArray.toSeq) + partitions += newPartition + } + currentFiles.clear() + currentSize = 0 + } + + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + // Assign files to partitions using "Next Fit Decreasing" + partitionedFiles.foreach { file => + if (currentSize + file.length > maxSplitBytes) { + closePartition() + } + // Add the given file to the current partition. + currentSize += file.length + openCostInBytes + currentFiles += file + } + closePartition() + partitions + } + + def maxSplitBytes( + sparkSession: SparkSession, + selectedPartitions: Seq[PartitionDirectory]): Long = { + val defaultMaxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + val defaultParallelism = sparkSession.sparkContext.defaultParallelism + val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum + val bytesPerCore = totalBytes / defaultParallelism + + Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index ffea33c08ef9..d92ea2e06895 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -29,6 +29,7 @@ import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.QueryExecutionException +import org.apache.spark.sql.sources.v2.reader.InputPartition import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.NextIterator @@ -53,12 +54,6 @@ case class PartitionedFile( } } -/** - * A collection of file blocks that should be read as a single task - * (possibly from multiple partitioned directories). - */ -case class FilePartition(index: Int, files: Seq[PartitionedFile]) extends RDDPartition - /** * An RDD that scans a list of file partitions. */ @@ -216,21 +211,6 @@ class FileScanRDD( override protected def getPartitions: Array[RDDPartition] = filePartitions.toArray override protected def getPreferredLocations(split: RDDPartition): Seq[String] = { - val files = split.asInstanceOf[FilePartition].files - - // Computes total number of bytes can be retrieved from each host. - val hostToNumBytes = mutable.HashMap.empty[String, Long] - files.foreach { file => - file.locations.filter(_ != "localhost").foreach { host => - hostToNumBytes(host) = hostToNumBytes.getOrElse(host, 0L) + file.length - } - } - - // Takes the first 3 hosts with the most data to be retrieved - hostToNumBytes.toSeq.sortBy { - case (host, numBytes) => numBytes - }.reverse.take(3).map { - case (host, numBytes) => host - } + split.asInstanceOf[FilePartition].preferredLocations() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index b2f73b7f8d1f..d278802e6c9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -52,28 +52,12 @@ case class HadoopFsRelation( override def sqlContext: SQLContext = sparkSession.sqlContext - private def getColName(f: StructField): String = { - if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { - f.name - } else { - f.name.toLowerCase(Locale.ROOT) - } - } - - val overlappedPartCols = mutable.Map.empty[String, StructField] - partitionSchema.foreach { partitionField => - if (dataSchema.exists(getColName(_) == getColName(partitionField))) { - overlappedPartCols += getColName(partitionField) -> partitionField - } - } - // When data and partition schemas have overlapping columns, the output // schema respects the order of the data schema for the overlapping columns, and it // respects the data types of the partition schema. - val schema: StructType = { - StructType(dataSchema.map(f => overlappedPartCols.getOrElse(getColName(f), f)) ++ - partitionSchema.filterNot(f => overlappedPartCols.contains(getColName(f)))) - } + val (schema: StructType, overlappedPartCols: Map[String, StructField]) = + PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, + partitionSchema, sparkSession.sessionState.conf.caseSensitiveAnalysis) def partitionSchemaOption: Option[StructType] = if (partitionSchema.isEmpty) None else Some(partitionSchema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 6458b65466fb..a2e08180cc50 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -21,18 +21,18 @@ import java.lang.{Double => JDouble, Long => JLong} import java.math.{BigDecimal => JBigDecimal} import java.util.{Locale, TimeZone} +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.Try import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCoercion} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Cast, Literal} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -59,14 +59,14 @@ object PartitionSpec { object PartitioningUtils { + val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]" + private[datasources] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { require(columnNames.size == literals.size) } - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.DEFAULT_PARTITION_NAME - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName + import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.{escapePathName, unescapePathName, DEFAULT_PARTITION_NAME} /** * Given a group of qualified paths, tries to parse them and returns a partition specification. @@ -122,10 +122,12 @@ object PartitioningUtils { Map.empty[String, DataType] } + val dateFormatter = DateFormatter() + val timestampFormatter = TimestampFormatter(timestampPartitionPattern, timeZone) // First, we need to parse every partition's path and see if we can find partition values. val (partitionValues, optDiscoveredBasePaths) = paths.map { path => parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, - validatePartitionColumns, timeZone) + validatePartitionColumns, timeZone, dateFormatter, timestampFormatter) }.unzip // We create pairs of (path -> path's partition value) here @@ -208,7 +210,9 @@ object PartitioningUtils { basePaths: Set[Path], userSpecifiedDataTypes: Map[String, DataType], validatePartitionColumns: Boolean, - timeZone: TimeZone): (Option[PartitionValues], Option[Path]) = { + timeZone: TimeZone, + dateFormatter: DateFormatter, + timestampFormatter: TimestampFormatter): (Option[PartitionValues], Option[Path]) = { val columns = ArrayBuffer.empty[(String, Literal)] // Old Hadoop versions don't have `Path.isRoot` var finished = path.getParent == null @@ -230,7 +234,7 @@ object PartitioningUtils { // Once we get the string, we try to parse it and find the partition column and value. val maybeColumn = parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes, - validatePartitionColumns, timeZone) + validatePartitionColumns, timeZone, dateFormatter, timestampFormatter) maybeColumn.foreach(columns += _) // Now, we determine if we should stop. @@ -265,7 +269,9 @@ object PartitioningUtils { typeInference: Boolean, userSpecifiedDataTypes: Map[String, DataType], validatePartitionColumns: Boolean, - timeZone: TimeZone): Option[(String, Literal)] = { + timeZone: TimeZone, + dateFormatter: DateFormatter, + timestampFormatter: TimestampFormatter): Option[(String, Literal)] = { val equalSignIndex = columnSpec.indexOf('=') if (equalSignIndex == -1) { None @@ -280,7 +286,12 @@ object PartitioningUtils { // SPARK-26188: if user provides corresponding column schema, get the column value without // inference, and then cast it as user specified data type. val dataType = userSpecifiedDataTypes(columnName) - val columnValueLiteral = inferPartitionColumnValue(rawColumnValue, false, timeZone) + val columnValueLiteral = inferPartitionColumnValue( + rawColumnValue, + false, + timeZone, + dateFormatter, + timestampFormatter) val columnValue = columnValueLiteral.eval() val castedValue = Cast(columnValueLiteral, dataType, Option(timeZone.getID)).eval() if (validatePartitionColumns && columnValue != null && castedValue == null) { @@ -289,7 +300,12 @@ object PartitioningUtils { } Literal.create(castedValue, dataType) } else { - inferPartitionColumnValue(rawColumnValue, typeInference, timeZone) + inferPartitionColumnValue( + rawColumnValue, + typeInference, + timeZone, + dateFormatter, + timestampFormatter) } Some(columnName -> literal) } @@ -442,7 +458,9 @@ object PartitioningUtils { private[datasources] def inferPartitionColumnValue( raw: String, typeInference: Boolean, - timeZone: TimeZone): Literal = { + timeZone: TimeZone, + dateFormatter: DateFormatter, + timestampFormatter: TimestampFormatter): Literal = { val decimalTry = Try { // `BigDecimal` conversion can fail when the `field` is not a form of number. val bigDecimal = new JBigDecimal(raw) @@ -457,7 +475,7 @@ object PartitioningUtils { val dateTry = Try { // try and parse the date, if no exception occurs this is a candidate to be resolved as // DateType - DateTimeUtils.getThreadLocalDateFormat(DateTimeUtils.defaultTimeZone()).parse(raw) + dateFormatter.parse(raw) // SPARK-23436: Casting the string to date may still return null if a bad Date is provided. // This can happen since DateFormat.parse may not use the entire text of the given string: // so if there are extra-characters after the date, it returns correctly. @@ -474,7 +492,7 @@ object PartitioningUtils { val unescapedRaw = unescapePathName(raw) // try and parse the date, if no exception occurs this is a candidate to be resolved as // TimestampType - DateTimeUtils.getThreadLocalTimestampFormat(timeZone).parse(unescapedRaw) + timestampFormatter.parse(unescapedRaw) // SPARK-23436: see comment for date val timestampValue = Cast(Literal(unescapedRaw), TimestampType, Some(timeZone.getID)).eval() // Disallow TimestampType if the cast returned null @@ -539,6 +557,35 @@ object PartitioningUtils { }).asNullable } + def mergeDataAndPartitionSchema( + dataSchema: StructType, + partitionSchema: StructType, + caseSensitive: Boolean): (StructType, Map[String, StructField]) = { + val overlappedPartCols = mutable.Map.empty[String, StructField] + partitionSchema.foreach { partitionField => + val partitionFieldName = getColName(partitionField, caseSensitive) + if (dataSchema.exists(getColName(_, caseSensitive) == partitionFieldName)) { + overlappedPartCols += partitionFieldName -> partitionField + } + } + + // When data and partition schemas have overlapping columns, the output + // schema respects the order of the data schema for the overlapping columns, and it + // respects the data types of the partition schema. + val fullSchema = + StructType(dataSchema.map(f => overlappedPartCols.getOrElse(getColName(f, caseSensitive), f)) ++ + partitionSchema.filterNot(f => overlappedPartCols.contains(getColName(f, caseSensitive)))) + (fullSchema, overlappedPartCols.toMap) + } + + def getColName(f: StructField, caseSensitive: Boolean): String = { + if (caseSensitive) { + f.name + } else { + f.name.toLowerCase(Locale.ROOT) + } + } + private def columnNameEquality(caseSensitive: Boolean): (String, String) => Boolean = { if (caseSensitive) { org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 51c385e25bee..13ed105004d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -26,7 +26,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.jdbc.JdbcDialects import org.apache.spark.sql.sources._ @@ -185,10 +185,11 @@ private[sql] object JDBCRelation extends Logging { columnType: DataType, timeZoneId: String): String = { def dateTimeToString(): String = { - val timeZone = DateTimeUtils.getTimeZone(timeZoneId) val dateTimeStr = columnType match { - case DateType => DateTimeUtils.dateToString(value.toInt, timeZone) - case TimestampType => DateTimeUtils.timestampToString(value, timeZone) + case DateType => DateFormatter().format(value.toInt) + case TimestampType => + val timestampFormatter = TimestampFormatter(DateTimeUtils.getTimeZone(timeZoneId)) + DateTimeUtils.timestampToString(timestampFormatter, value) } s"'$dateTimeStr'" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala new file mode 100644 index 000000000000..79e4c62a32bd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.noop + +import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.types.StructType + +/** + * This is no-op datasource. It does not do anything besides consuming its input. + * This can be useful for benchmarking or to cache data without any additional overhead. + */ +class NoopDataSource + extends DataSourceV2 + with TableProvider + with DataSourceRegister { + + override def shortName(): String = "noop" + override def getTable(options: DataSourceOptions): Table = NoopTable +} + +private[noop] object NoopTable extends Table with SupportsBatchWrite { + override def newWriteBuilder(options: DataSourceOptions): WriteBuilder = NoopWriteBuilder + override def name(): String = "noop-table" + override def schema(): StructType = new StructType() +} + +private[noop] object NoopWriteBuilder extends WriteBuilder with SupportsSaveMode { + override def buildForBatch(): BatchWrite = NoopBatchWrite + override def mode(mode: SaveMode): WriteBuilder = this +} + +private[noop] object NoopBatchWrite extends BatchWrite { + override def createBatchWriterFactory(): DataWriterFactory = NoopWriterFactory + override def commit(messages: Array[WriterCommitMessage]): Unit = {} + override def abort(messages: Array[WriterCommitMessage]): Unit = {} +} + +private[noop] object NoopWriterFactory extends DataWriterFactory { + override def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = NoopWriter +} + +private[noop] object NoopWriter extends DataWriter[InternalRow] { + override def write(record: InternalRow): Unit = {} + override def commit(): WriterCommitMessage = null + override def abort(): Unit = {} +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 0a64981b421c..cd2a68a53bab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -82,22 +82,24 @@ private[sql] object OrcFilters { */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap - - // First, tries to convert each filter individually to see whether it's convertible, and then - // collect all convertible ones to build the final `SearchArgument`. - val convertibleFilters = for { - filter <- filters - _ <- buildSearchArgument(dataTypeMap, filter, newBuilder) - } yield filter - for { // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(convertibleFilters) + conjunction <- buildTree(convertibleFilters(schema, dataTypeMap, filters)) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate builder <- buildSearchArgument(dataTypeMap, conjunction, newBuilder) } yield builder.build() } + def convertibleFilters( + schema: StructType, + dataTypeMap: Map[String, DataType], + filters: Seq[Filter]): Seq[Filter] = { + for { + filter <- filters + _ <- buildSearchArgument(dataTypeMap, filter, newBuilder()) + } yield filter + } + /** * Return true if this is a searchable type in ORC. * Both CharType and VarcharType are cleaned at AstBuilder. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala index 91080b15727d..840fcae8c691 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala @@ -116,10 +116,28 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { // For example, for a query `SELECT name.first FROM contacts WHERE name IS NOT NULL`, // we don't need to read nested fields of `name` struct other than `first` field. val (rootFields, optRootFields) = (projectionRootFields ++ filterRootFields) - .distinct.partition(_.contentAccessed) + .distinct.partition(!_.prunedIfAnyChildAccessed) optRootFields.filter { opt => - !rootFields.exists(_.field.name == opt.field.name) + !rootFields.exists { root => + root.field.name == opt.field.name && { + // Checking if current optional root field can be pruned. + // For each required root field, we merge it with the optional root field: + // 1. If this optional root field has nested fields and any nested field of it is used + // in the query, the merged field type must equal to the optional root field type. + // We can prune this optional root field. For example, for optional root field + // `struct>`, if its field + // `struct>` is used, we don't need to add this optional + // root field. + // 2. If this optional root field has no nested fields, the merged field type equals + // to the optional root field only if they are the same. If they are, we can prune + // this optional root field too. + val rootFieldType = StructType(Array(root.field)) + val optFieldType = StructType(Array(opt.field)) + val merged = optFieldType.merge(rootFieldType) + merged.sameType(optFieldType) + } + } } ++ rootFields } @@ -213,11 +231,11 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { // don't actually use any nested fields. These root field accesses might be excluded later // if there are any nested fields accesses in the query plan. case IsNotNull(SelectedField(field)) => - RootField(field, derivedFromAtt = false, contentAccessed = false) :: Nil + RootField(field, derivedFromAtt = false, prunedIfAnyChildAccessed = true) :: Nil case IsNull(SelectedField(field)) => - RootField(field, derivedFromAtt = false, contentAccessed = false) :: Nil + RootField(field, derivedFromAtt = false, prunedIfAnyChildAccessed = true) :: Nil case IsNotNull(_: Attribute) | IsNull(_: Attribute) => - expr.children.flatMap(getRootFields).map(_.copy(contentAccessed = false)) + expr.children.flatMap(getRootFields).map(_.copy(prunedIfAnyChildAccessed = true)) case _ => expr.children.flatMap(getRootFields) } @@ -271,9 +289,9 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { /** * This represents a "root" schema field (aka top-level, no-parent). `field` is the * `StructField` for field name and datatype. `derivedFromAtt` indicates whether it - * was derived from an attribute or had a proper child. `contentAccessed` means whether - * it was accessed with its content by the expressions refer it. + * was derived from an attribute or had a proper child. `prunedIfAnyChildAccessed` means + * whether this root field can be pruned if any of child field is used in the query. */ private case class RootField(field: StructField, derivedFromAtt: Boolean, - contentAccessed: Boolean = true) + prunedIfAnyChildAccessed: Boolean = false) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 7bf2b8bff373..632157818434 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -21,46 +21,51 @@ import java.util.UUID import scala.collection.JavaConverters._ -import org.apache.spark.sql.{AnalysisException, SaveMode} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport +import org.apache.spark.sql.sources.v2.writer._ import org.apache.spark.sql.types.StructType /** - * A logical plan representing a data source v2 scan. + * A logical plan representing a data source v2 table. * - * @param source An instance of a [[DataSourceV2]] implementation. - * @param options The options for this scan. Used to create fresh [[BatchWriteSupport]]. - * @param userSpecifiedSchema The user-specified schema for this scan. + * @param table The table that this relation represents. + * @param options The options for this table operation. It's used to create fresh [[ScanBuilder]] + * and [[WriteBuilder]]. */ case class DataSourceV2Relation( - // TODO: remove `source` when we finish API refactor for write. - source: TableProvider, - table: SupportsBatchRead, + table: Table, output: Seq[AttributeReference], - options: Map[String, String], - userSpecifiedSchema: Option[StructType] = None) + options: Map[String, String]) extends LeafNode with MultiInstanceRelation with NamedRelation { - import DataSourceV2Relation._ - override def name: String = table.name() override def simpleString(maxFields: Int): String = { s"RelationV2${truncatedString(output, "[", ", ", "]", maxFields)} $name" } - def newWriteSupport(): BatchWriteSupport = source.createWriteSupport(options, schema) + def newScanBuilder(): ScanBuilder = table match { + case s: SupportsBatchRead => + val dsOptions = new DataSourceOptions(options.asJava) + s.newScanBuilder(dsOptions) + case _ => throw new AnalysisException(s"Table is not readable: ${table.name()}") + } + - def newScanBuilder(): ScanBuilder = { - val dsOptions = new DataSourceOptions(options.asJava) - table.newScanBuilder(dsOptions) + + def newWriteBuilder(schema: StructType): WriteBuilder = table match { + case s: SupportsBatchWrite => + val dsOptions = new DataSourceOptions(options.asJava) + s.newWriteBuilder(dsOptions) + .withQueryId(UUID.randomUUID().toString) + .withInputDataSchema(schema) + case _ => throw new AnalysisException(s"Table is not writable: ${table.name()}") } override def computeStats(): Statistics = { @@ -126,52 +131,8 @@ case class StreamingDataSourceV2Relation( } object DataSourceV2Relation { - private implicit class SourceHelpers(source: DataSourceV2) { - def asWriteSupportProvider: BatchWriteSupportProvider = { - source match { - case provider: BatchWriteSupportProvider => - provider - case _ => - throw new AnalysisException(s"Data source is not writable: $name") - } - } - - def name: String = { - source match { - case registered: DataSourceRegister => - registered.shortName() - case _ => - source.getClass.getSimpleName - } - } - - def createWriteSupport( - options: Map[String, String], - schema: StructType): BatchWriteSupport = { - asWriteSupportProvider.createBatchWriteSupport( - UUID.randomUUID().toString, - schema, - SaveMode.Append, - new DataSourceOptions(options.asJava)).get - } - } - - def create( - provider: TableProvider, - table: SupportsBatchRead, - options: Map[String, String], - userSpecifiedSchema: Option[StructType] = None): DataSourceV2Relation = { + def create(table: Table, options: Map[String, String]): DataSourceV2Relation = { val output = table.schema().toAttributes - DataSourceV2Relation(provider, table, output, options, userSpecifiedSchema) - } - - // TODO: remove this when we finish API refactor for write. - def createRelationForWrite( - source: DataSourceV2, - options: Map[String, String]): DataSourceV2Relation = { - val provider = source.asInstanceOf[TableProvider] - val dsOptions = new DataSourceOptions(options.asJava) - val table = provider.getTable(dsOptions) - create(provider, table.asInstanceOf[SupportsBatchRead], options) + DataSourceV2Relation(table, output, options) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 2e26fce880b6..79540b024621 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable -import org.apache.spark.sql.{sources, Strategy} +import org.apache.spark.sql.{sources, AnalysisException, SaveMode, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, Repartition} @@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport +import org.apache.spark.sql.sources.v2.writer.SupportsSaveMode object DataSourceV2Strategy extends Strategy { @@ -110,7 +111,7 @@ object DataSourceV2Strategy extends Strategy { val (scan, output) = pruneColumns(scanBuilder, relation, project ++ postScanFilters) logInfo( s""" - |Pushing operators to ${relation.source.getClass} + |Pushing operators to ${relation.name} |Pushed Filters: ${pushedFilters.mkString(", ")} |Post-Scan Filters: ${postScanFilters.mkString(",")} |Output: ${output.mkString(", ")} @@ -136,7 +137,14 @@ object DataSourceV2Strategy extends Strategy { WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil case AppendData(r: DataSourceV2Relation, query, _) => - WriteToDataSourceV2Exec(r.newWriteSupport(), planLater(query)) :: Nil + val writeBuilder = r.newWriteBuilder(query.schema) + writeBuilder match { + case s: SupportsSaveMode => + val write = s.mode(SaveMode.Append).buildForBatch() + assert(write != null) + WriteToDataSourceV2Exec(write, planLater(query)) :: Nil + case _ => throw new AnalysisException(s"data source ${r.name} does not support SaveMode") + } case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala new file mode 100644 index 000000000000..b177d15e1fe3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import java.io.IOException + +import org.apache.spark.sql.sources.v2.reader.PartitionReader + +/** + * A [[PartitionReader]] with empty output. + */ +class EmptyPartitionReader[T] extends PartitionReader[T] { + override def next(): Boolean = false + + override def get(): T = + throw new IOException("No records should be returned from EmptyDataReader") + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala new file mode 100644 index 000000000000..a0c932cbb0e0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, SupportsBatchRead, TableProvider} +import org.apache.spark.sql.types.StructType + +/** + * A base interface for data source v2 implementations of the built-in file-based data sources. + */ +trait FileDataSourceV2 extends TableProvider with DataSourceRegister { + /** + * Returns a V1 [[FileFormat]] class of the same file data source. + * This is a solution for the following cases: + * 1. File datasource V2 implementations cause regression. Users can disable the problematic data + * source via SQL configuration and fall back to FileFormat. + * 2. Catalog support is required, which is still under development for data source V2. + */ + def fallBackFileFormat: Class[_ <: FileFormat] + + lazy val sparkSession = SparkSession.active + + def getFileIndex( + options: DataSourceOptions, + userSpecifiedSchema: Option[StructType]): PartitioningAwareFileIndex = { + val filePaths = options.paths() + val hadoopConf = + sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) + val rootPathsSpecified = DataSource.checkAndGlobPathIfNecessary(filePaths, hadoopConf, + checkEmptyGlobPath = true, checkFilesExist = options.checkFilesExist()) + val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) + new InMemoryFileIndex(sparkSession, rootPathsSpecified, + options.asMap().asScala.toMap, userSpecifiedSchema, fileStatusCache) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala new file mode 100644 index 000000000000..d76d69dba31d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import java.io.{FileNotFoundException, IOException} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.PartitionReader + +class FilePartitionReader[T](readers: Iterator[PartitionedFileReader[T]]) + extends PartitionReader[T] with Logging { + private var currentReader: PartitionedFileReader[T] = null + + private val sqlConf = SQLConf.get + private def ignoreMissingFiles = sqlConf.ignoreMissingFiles + private def ignoreCorruptFiles = sqlConf.ignoreCorruptFiles + + override def next(): Boolean = { + if (currentReader == null) { + if (readers.hasNext) { + if (ignoreMissingFiles || ignoreCorruptFiles) { + try { + currentReader = readers.next() + logInfo(s"Reading file $currentReader") + } catch { + case e: FileNotFoundException if ignoreMissingFiles => + logWarning(s"Skipped missing file: $currentReader", e) + currentReader = null + return false + // Throw FileNotFoundException even if `ignoreCorruptFiles` is true + case e: FileNotFoundException if !ignoreMissingFiles => throw e + case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => + logWarning( + s"Skipped the rest of the content in the corrupted file: $currentReader", e) + currentReader = null + return false + } + } else { + currentReader = readers.next() + logInfo(s"Reading file $currentReader") + } + } else { + return false + } + } + if (currentReader.next()) { + true + } else { + close() + currentReader = null + next() + } + } + + override def get(): T = currentReader.get() + + override def close(): Unit = { + if (currentReader != null) { + currentReader.close() + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala new file mode 100644 index 000000000000..101a70ee92ce --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.vectorized.ColumnarBatch + +abstract class FilePartitionReaderFactory extends PartitionReaderFactory { + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + assert(partition.isInstanceOf[FilePartition]) + val filePartition = partition.asInstanceOf[FilePartition] + val iter = filePartition.files.toIterator.map { file => + new PartitionedFileReader(file, buildReader(file)) + } + new FilePartitionReader[InternalRow](iter) + } + + override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { + assert(partition.isInstanceOf[FilePartition]) + val filePartition = partition.asInstanceOf[FilePartition] + val iter = filePartition.files.toIterator.map { file => + new PartitionedFileReader(file, buildColumnarReader(file)) + } + new FilePartitionReader[ColumnarBatch](iter) + } + + def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] + + def buildColumnarReader(partitionedFile: PartitionedFile): PartitionReader[ColumnarBatch] = { + throw new UnsupportedOperationException("Cannot create columnar reader.") + } +} + +// A compound class for combining file and its corresponding reader. +private[v2] class PartitionedFileReader[T]( + file: PartitionedFile, + reader: PartitionReader[T]) extends PartitionReader[T] { + override def next(): Boolean = reader.next() + + override def get(): T = reader.get() + + override def close(): Unit = reader.close() + + override def toString: String = file.toString +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala new file mode 100644 index 000000000000..3615b15be6fd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.PartitionedFileUtil +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, Scan} +import org.apache.spark.sql.types.StructType + +abstract class FileScan( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex) extends Scan with Batch { + /** + * Returns whether a file with `path` could be split or not. + */ + def isSplitable(path: Path): Boolean = { + false + } + + protected def partitions: Seq[FilePartition] = { + val selectedPartitions = fileIndex.listFiles(Seq.empty, Seq.empty) + val maxSplitBytes = FilePartition.maxSplitBytes(sparkSession, selectedPartitions) + val splitFiles = selectedPartitions.flatMap { partition => + partition.files.flatMap { file => + val filePath = file.getPath + PartitionedFileUtil.splitFiles( + sparkSession = sparkSession, + file = file, + filePath = filePath, + isSplitable = isSplitable(filePath), + maxSplitBytes = maxSplitBytes, + partitionValues = partition.values + ) + }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) + } + FilePartition.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) + } + + override def planInputPartitions(): Array[InputPartition] = { + partitions.toArray + } + + override def toBatch: Batch = this +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala new file mode 100644 index 000000000000..5dd343ba44b6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.sources.v2.reader.{ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} +import org.apache.spark.sql.types.StructType + +abstract class FileScanBuilder(schema: StructType) + extends ScanBuilder + with SupportsPushDownRequiredColumns + with SupportsPushDownFilters { + protected var readSchema = schema + + override def pruneColumns(requiredSchema: StructType): Unit = { + this.readSchema = requiredSchema + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala new file mode 100644 index 000000000000..b1786541a805 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources.v2.{SupportsBatchRead, Table} +import org.apache.spark.sql.types.StructType + +abstract class FileTable( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + userSpecifiedSchema: Option[StructType]) extends Table with SupportsBatchRead { + def getFileIndex: PartitioningAwareFileIndex = this.fileIndex + + lazy val dataSchema: StructType = userSpecifiedSchema.orElse { + inferSchema(fileIndex.allFiles()) + }.getOrElse { + throw new AnalysisException( + s"Unable to infer schema for $name. It must be specified manually.") + }.asNullable + + override def schema(): StructType = { + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, + fileIndex.partitionSchema, caseSensitive)._1 + } + + /** + * When possible, this method should return the schema of the given `files`. When the format + * does not support inference, or no valid files are given should return None. In these cases + * Spark will require that user specify the schema manually. + */ + def inferSchema(files: Seq[FileStatus]): Option[StructType] +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala new file mode 100644 index 000000000000..ff78ef3220c1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.hadoop.mapreduce.RecordReader + +import org.apache.spark.sql.sources.v2.reader.PartitionReader + +class PartitionRecordReader[T]( + private[this] var rowReader: RecordReader[_, T]) extends PartitionReader[T] { + override def next(): Boolean = rowReader.nextKeyValue() + + override def get(): T = rowReader.getCurrentValue + + override def close(): Unit = rowReader.close() +} + +class PartitionRecordReaderWithProject[X, T]( + private[this] var rowReader: RecordReader[_, X], + project: X => T) extends PartitionReader[T] { + override def next(): Boolean = rowReader.nextKeyValue() + + override def get(): T = project(rowReader.getCurrentValue) + + override def close(): Unit = rowReader.close() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index d7e20eed4cbc..406fb8c3a383 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.{LongAccumulator, Utils} * specific logical plans, like [[org.apache.spark.sql.catalyst.plans.logical.AppendData]]. */ @deprecated("Use specific logical plans like AppendData instead", "2.4.0") -case class WriteToDataSourceV2(writeSupport: BatchWriteSupport, query: LogicalPlan) +case class WriteToDataSourceV2(batchWrite: BatchWrite, query: LogicalPlan) extends LogicalPlan { override def children: Seq[LogicalPlan] = Seq(query) override def output: Seq[Attribute] = Nil @@ -44,7 +44,7 @@ case class WriteToDataSourceV2(writeSupport: BatchWriteSupport, query: LogicalPl /** * The physical plan for writing data into data source v2. */ -case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: SparkPlan) +case class WriteToDataSourceV2Exec(batchWrite: BatchWrite, query: SparkPlan) extends UnaryExecNode { var commitProgress: Option[StreamWriterCommitProgress] = None @@ -53,13 +53,13 @@ case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: Spark override def output: Seq[Attribute] = Nil override protected def doExecute(): RDD[InternalRow] = { - val writerFactory = writeSupport.createBatchWriterFactory() - val useCommitCoordinator = writeSupport.useCommitCoordinator + val writerFactory = batchWrite.createBatchWriterFactory() + val useCommitCoordinator = batchWrite.useCommitCoordinator val rdd = query.execute() val messages = new Array[WriterCommitMessage](rdd.partitions.length) val totalNumRowsAccumulator = new LongAccumulator() - logInfo(s"Start processing data source write support: $writeSupport. " + + logInfo(s"Start processing data source write support: $batchWrite. " + s"The input RDD has ${messages.length} partitions.") try { @@ -72,26 +72,26 @@ case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: Spark val commitMessage = result.writerCommitMessage messages(index) = commitMessage totalNumRowsAccumulator.add(result.numRows) - writeSupport.onDataWriterCommit(commitMessage) + batchWrite.onDataWriterCommit(commitMessage) } ) - logInfo(s"Data source write support $writeSupport is committing.") - writeSupport.commit(messages) - logInfo(s"Data source write support $writeSupport committed.") + logInfo(s"Data source write support $batchWrite is committing.") + batchWrite.commit(messages) + logInfo(s"Data source write support $batchWrite committed.") commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value)) } catch { case cause: Throwable => - logError(s"Data source write support $writeSupport is aborting.") + logError(s"Data source write support $batchWrite is aborting.") try { - writeSupport.abort(messages) + batchWrite.abort(messages) } catch { case t: Throwable => - logError(s"Data source write support $writeSupport failed to abort.") + logError(s"Data source write support $batchWrite failed to abort.") cause.addSuppressed(t) throw new SparkException("Writing job failed.", cause) } - logError(s"Data source write support $writeSupport aborted.") + logError(s"Data source write support $batchWrite aborted.") cause match { // Only wrap non fatal exceptions. case NonFatal(e) => throw new SparkException("Writing job aborted.", e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala new file mode 100644 index 000000000000..db1f2f793422 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.orc + +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.sources.v2.{DataSourceOptions, Table} +import org.apache.spark.sql.types.StructType + +class OrcDataSourceV2 extends FileDataSourceV2 { + + override def fallBackFileFormat: Class[_ <: FileFormat] = classOf[OrcFileFormat] + + override def shortName(): String = "orc" + + private def getTableName(options: DataSourceOptions): String = { + shortName() + ":" + options.paths().mkString(";") + } + + override def getTable(options: DataSourceOptions): Table = { + val tableName = getTableName(options) + val fileIndex = getFileIndex(options, None) + OrcTable(tableName, sparkSession, fileIndex, None) + } + + override def getTable(options: DataSourceOptions, schema: StructType): Table = { + val tableName = getTableName(options) + val fileIndex = getFileIndex(options, Some(schema)) + OrcTable(tableName, sparkSession, fileIndex, Some(schema)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala new file mode 100644 index 000000000000..f6fc0ca83908 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.orc + +import java.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.orc.{OrcConf, OrcFile} +import org.apache.orc.mapred.OrcStruct +import org.apache.orc.mapreduce.OrcInputFormat + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.datasources.{PartitionedFile, PartitioningUtils} +import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcDeserializer, OrcUtils} +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader} +import org.apache.spark.sql.types.{AtomicType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + +/** + * A factory used to create Orc readers. + * + * @param sqlConf SQL configuration. + * @param broadcastedConf Broadcast serializable Hadoop Configuration. + * @param dataSchema Schema of orc files. + * @param partitionSchema Schema of partitions. + * @param readSchema Required schema in the batch scan. + */ +case class OrcPartitionReaderFactory( + sqlConf: SQLConf, + broadcastedConf: Broadcast[SerializableConfiguration], + dataSchema: StructType, + partitionSchema: StructType, + readSchema: StructType) extends FilePartitionReaderFactory { + private val isCaseSensitive = sqlConf.caseSensitiveAnalysis + private val capacity = sqlConf.orcVectorizedReaderBatchSize + + override def supportColumnarReads(partition: InputPartition): Boolean = { + sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && + readSchema.length <= sqlConf.wholeStageMaxNumFields && + readSchema.forall(_.dataType.isInstanceOf[AtomicType]) + } + + override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { + val conf = broadcastedConf.value.value + + val filePath = new Path(new URI(file.filePath)) + + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(filePath, readerOptions) + + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readSchema, reader, conf) + + if (requestedColIdsOrEmptyFile.isEmpty) { + new EmptyPartitionReader[InternalRow] + } else { + val requestedColIds = requestedColIdsOrEmptyFile.get + assert(requestedColIds.length == readSchema.length, + "[BUG] requested column IDs do not match required schema") + val taskConf = new Configuration(conf) + taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, + requestedColIds.filter(_ != -1).sorted.mkString(",")) + + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + + val requiredDataSchema = subtractSchema(readSchema, partitionSchema) + val orcRecordReader = new OrcInputFormat[OrcStruct] + .createRecordReader(fileSplit, taskAttemptContext) + + val fullSchema = requiredDataSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val deserializer = new OrcDeserializer(dataSchema, requiredDataSchema, requestedColIds) + + val projection = if (partitionSchema.length == 0) { + (value: OrcStruct) => unsafeProjection(deserializer.deserialize(value)) + } else { + val joinedRow = new JoinedRow() + (value: OrcStruct) => + unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues)) + } + new PartitionRecordReaderWithProject(orcRecordReader, projection) + } + } + + override def buildColumnarReader(file: PartitionedFile): PartitionReader[ColumnarBatch] = { + val conf = broadcastedConf.value.value + + val filePath = new Path(new URI(file.filePath)) + + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(filePath, readerOptions) + + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readSchema, reader, conf) + + if (requestedColIdsOrEmptyFile.isEmpty) { + new EmptyPartitionReader + } else { + val requestedColIds = requestedColIdsOrEmptyFile.get + assert(requestedColIds.length == readSchema.length, + "[BUG] requested column IDs do not match required schema") + val taskConf = new Configuration(conf) + taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, + requestedColIds.filter(_ != -1).sorted.mkString(",")) + + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + + val batchReader = new OrcColumnarBatchReader(capacity) + batchReader.initialize(fileSplit, taskAttemptContext) + val columnNameMap = partitionSchema.fields.map( + PartitioningUtils.getColName(_, isCaseSensitive)).zipWithIndex.toMap + val requestedPartitionColIds = readSchema.fields.map { field => + columnNameMap.getOrElse(PartitioningUtils.getColName(field, isCaseSensitive), -1) + } + + batchReader.initBatch( + reader.getSchema, + readSchema.fields, + requestedColIds, + requestedPartitionColIds, + file.partitionValues) + new PartitionRecordReader(batchReader) + } + } + + /** + * Returns a new StructType that is a copy of the original StructType, removing any items that + * also appear in other StructType. The order is preserved from the original StructType. + */ + private def subtractSchema(original: StructType, other: StructType): StructType = { + val otherNameSet = other.fields.map(PartitioningUtils.getColName(_, isCaseSensitive)).toSet + val fields = original.fields.filterNot { field => + otherNameSet.contains(PartitioningUtils.getColName(field, isCaseSensitive)) + } + + StructType(fields) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala new file mode 100644 index 000000000000..a792ad318b39 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.orc + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +case class OrcScan( + sparkSession: SparkSession, + hadoopConf: Configuration, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readSchema: StructType) extends FileScan(sparkSession, fileIndex) { + override def isSplitable(path: Path): Boolean = true + + override def createReaderFactory(): PartitionReaderFactory = { + val broadcastedConf = sparkSession.sparkContext.broadcast( + new SerializableConfiguration(hadoopConf)) + OrcPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, + dataSchema, fileIndex.partitionSchema, readSchema) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala new file mode 100644 index 000000000000..eb27bbd3abea --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2.orc + +import scala.collection.JavaConverters._ + +import org.apache.orc.mapreduce.OrcInputFormat + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.orc.OrcFilters +import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.sources.v2.reader.Scan +import org.apache.spark.sql.types.StructType + +case class OrcScanBuilder( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + schema: StructType, + dataSchema: StructType, + options: DataSourceOptions) extends FileScanBuilder(schema) { + lazy val hadoopConf = + sparkSession.sessionState.newHadoopConfWithOptions(options.asMap().asScala.toMap) + + override def build(): Scan = { + OrcScan(sparkSession, hadoopConf, fileIndex, dataSchema, readSchema) + } + + private var _pushedFilters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + if (sparkSession.sessionState.conf.orcFilterPushDown) { + OrcFilters.createFilter(schema, filters).foreach { f => + // The pushed filters will be set in `hadoopConf`. After that, we can simply use the + // changed `hadoopConf` in executors. + OrcInputFormat.setSearchArgument(hadoopConf, f, schema.fieldNames) + } + val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + _pushedFilters = OrcFilters.convertibleFilters(schema, dataTypeMap, filters).toArray + } + filters + } + + override def pushedFilters(): Array[Filter] = _pushedFilters +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala new file mode 100644 index 000000000000..719e757c33cb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.orc + +import org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.orc.OrcUtils +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.types.StructType + +case class OrcTable( + name: String, + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + userSpecifiedSchema: Option[StructType]) + extends FileTable(sparkSession, fileIndex, userSpecifiedSchema) { + override def newScanBuilder(options: DataSourceOptions): OrcScanBuilder = + new OrcScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) + + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = + OrcUtils.readSchema(sparkSession, files) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index a80673c705f1..5e46d925e375 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPar import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.joins.HashedRelation import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.util.{SparkFatalException, ThreadUtils} /** @@ -44,10 +44,10 @@ case class BroadcastExchangeExec( child: SparkPlan) extends Exchange { override lazy val metrics = Map( - "dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"), - "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)"), - "buildTime" -> SQLMetrics.createMetric(sparkContext, "time to build (ms)"), - "broadcastTime" -> SQLMetrics.createMetric(sparkContext, "time to broadcast (ms)")) + "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), + "collectTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to collect"), + "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build"), + "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) @@ -79,7 +79,7 @@ case class BroadcastExchangeExec( val (numRows, input) = child.executeCollectIterator() if (numRows >= 512000000) { throw new SparkException( - s"Cannot broadcast the table with more than 512 millions rows: $numRows rows") + s"Cannot broadcast the table with 512 million or more rows: $numRows rows") } val beforeBuild = System.nanoTime() @@ -157,5 +157,6 @@ case class BroadcastExchangeExec( object BroadcastExchangeExec { private[execution] val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128)) + ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", + SQLConf.get.getConf(StaticSQLConf.BROADCAST_EXCHANGE_MAX_THREAD_THREASHOLD))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index da7b0c6f43fb..16398e34bdeb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -21,6 +21,7 @@ import java.util.Random import java.util.function.Supplier import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleWriteMetricsReporter, ShuffleWriteProcessor} @@ -172,7 +173,7 @@ object ShuffleExchangeExec { val conf = SparkEnv.get.conf val shuffleManager = SparkEnv.get.shuffleManager val sortBasedShuffleOn = shuffleManager.isInstanceOf[SortShuffleManager] - val bypassMergeThreshold = conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + val bypassMergeThreshold = conf.get(config.SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD) val numParts = partitioner.numPartitions if (sortBasedShuffleOn) { if (numParts <= bypassMergeThreshold) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 1aef5f686426..5ee4c7ffb191 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{RowIterator, SparkPlan} @@ -63,9 +64,8 @@ trait HashJoin { protected lazy val (buildKeys, streamedKeys) = { require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), "Join keys from two sides should have same types") - val lkeys = HashJoin.rewriteKeyExpr(leftKeys).map(BindReferences.bindReference(_, left.output)) - val rkeys = HashJoin.rewriteKeyExpr(rightKeys) - .map(BindReferences.bindReference(_, right.output)) + val lkeys = bindReferences(HashJoin.rewriteKeyExpr(leftKeys), left.output) + val rkeys = bindReferences(HashJoin.rewriteKeyExpr(rightKeys), right.output) buildSide match { case BuildLeft => (lkeys, rkeys) case BuildRight => (rkeys, lkeys) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index a708926dd1f8..90abc84daa77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -413,7 +413,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap private def init(): Unit = { if (mm != null) { - require(capacity < 512000000, "Cannot broadcast more than 512 millions rows") + require(capacity < 512000000, "Cannot broadcast 512 million or more rows") var n = 1 while (n < capacity) n *= 2 ensureAcquireMemory(n * 2L * 8 + (1 << 20)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index d7d3f6d6078b..f829f07e8072 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans._ @@ -393,7 +394,7 @@ case class SortMergeJoinExec( input: Seq[Attribute]): Seq[ExprCode] = { ctx.INPUT_ROW = row ctx.currentVars = null - keys.map(BindReferences.bindReference(_, input).genCode(ctx)) + bindReferences(keys, input).map(_.genCode(ctx)) } private def copyKeys(ctx: CodegenContext, vars: Seq[ExprCode]): Seq[ExprCode] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala index 8628471fdb92..7b2ea9627a98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala @@ -117,7 +117,9 @@ class FileStreamSourceLog( val batches = (existedBatches ++ retrievedBatches).map(i => i._1 -> i._2.get).toArray.sortBy(_._1) - HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId) + if (startBatchId <= endBatchId) { + HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId) + } batches } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index bd0a46115ceb..62d524ff19b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -262,7 +262,8 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: object HDFSMetadataLog { /** - * Verify if batchIds are continuous and between `startId` and `endId`. + * Verify if batchIds are continuous and between `startId` and `endId` (both inclusive and + * startId assumed to be <= endId). * * @param batchIds the sorted ids to verify. * @param startId the start id. If it's set, batchIds should start with this id. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 38ecb0dd12da..db1bf32a156c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, StreamWriterCommitProgress, WriteToDataSourceV2, WriteToDataSourceV2Exec} -import org.apache.spark.sql.execution.streaming.sources.{MicroBatchWritSupport, RateControlMicroBatchReadSupport} +import org.apache.spark.sql.execution.streaming.sources.{MicroBatchWrite, RateControlMicroBatchReadSupport} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} @@ -515,7 +515,7 @@ class MicroBatchExecution( newAttributePlan.schema, outputMode, new DataSourceOptions(extraOptions.asJava)) - WriteToDataSourceV2(new MicroBatchWritSupport(currentBatchId, writer), newAttributePlan) + WriteToDataSourceV2(new MicroBatchWrite(currentBatchId, writer), newAttributePlan) case _ => throw new IllegalArgumentException(s"unknown sink type for $sink") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 83824f40ab90..90f7b477103a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -181,6 +181,9 @@ abstract class StreamExecution( lazy val streamMetrics = new MetricsReporter( this, s"spark.streaming.${Option(name).getOrElse(id)}") + /** Isolated spark session to run the batches with. */ + private val sparkSessionForStream = sparkSession.cloneSession() + /** * The thread that runs the micro-batches of this stream. Note that this thread must be * [[org.apache.spark.util.UninterruptibleThread]] to workaround KAFKA-1894: interrupting a @@ -270,8 +273,6 @@ abstract class StreamExecution( // force initialization of the logical plan so that the sources can be created logicalPlan - // Isolated spark session to run the batches with. - val sparkSessionForStream = sparkSession.cloneSession() // Adaptive execution can change num shuffle partitions, disallow sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") // Disable cost-based join optimization as we do not want stateful operations to be rearranged diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala similarity index 84% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala index 9f88416871f8..143235efee81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala @@ -18,16 +18,15 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.{BatchWriteSupport, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} /** - * A [[BatchWriteSupport]] used to hook V2 stream writers into a microbatch plan. It implements + * A [[BatchWrite]] used to hook V2 stream writers into a microbatch plan. It implements * the non-streaming interface, forwarding the epoch ID determined at construction to a wrapped * streaming write support. */ -class MicroBatchWritSupport(eppchId: Long, val writeSupport: StreamingWriteSupport) - extends BatchWriteSupport { +class MicroBatchWrite(eppchId: Long, val writeSupport: StreamingWriteSupport) extends BatchWrite { override def commit(messages: Array[WriterCommitMessage]): Unit = { writeSupport.commit(eppchId, messages) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala index ac3c71cc222b..fd4cb444ce58 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala @@ -21,12 +21,12 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.{BatchWriteSupport, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.sources.v2.writer.streaming.StreamingDataWriterFactory /** * A simple [[DataWriterFactory]] whose tasks just pack rows into the commit message for delivery - * to a [[BatchWriteSupport]] on the driver. + * to a [[BatchWrite]] on the driver. * * Note that, because it sends all rows to the driver, this factory will generally be unsuitable * for production-quality sinks. It's intended for use in tests. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index c11af345b024..d689a6f3c981 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -79,8 +79,8 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), "numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows"), - "allUpdatesTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "total time to update rows"), - "allRemovalsTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "total time to remove rows"), + "allUpdatesTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "time to update"), + "allRemovalsTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "time to remove"), "commitTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "time to commit changes"), "stateMemory" -> SQLMetrics.createSizeMetric(sparkContext, "memory used by state") ) ++ stateStoreCustomMetrics diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala index a5601899ea2d..d5f2ffa5573a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala @@ -21,6 +21,7 @@ import java.util import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray @@ -103,9 +104,8 @@ final class OffsetWindowFunctionFrame( private[this] val projection = { // Collect the expressions and bind them. val inputAttrs = inputSchema.map(_.withNullability(true)) - val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map { e => - BindReferences.bindReference(e.input, inputAttrs) - } + val boundExpressions = Seq.fill(ordinal)(NoOp) ++ bindReferences( + expressions.toSeq.map(_.input), inputAttrs) // Create the projection. newMutableProjection(boundExpressions, Nil).target(target) @@ -114,7 +114,7 @@ final class OffsetWindowFunctionFrame( /** Create the projection used when the offset row DOES NOT exists. */ private[this] val fillDefaultValue = { // Collect the expressions and bind them. - val inputAttrs = inputSchema.map(_.withNullability(true)) + val inputAttrs: AttributeSeq = inputSchema.map(_.withNullability(true)) val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map { e => if (e.default == null || e.default.foldable && e.default.eval() == null) { // The default value is null. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 319c2649592f..a605dc640dc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -159,6 +159,7 @@ abstract class BaseSessionStateBuilder( override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: + new FallbackOrcDataSourceV2(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index f8d2bc8e0f13..5be45c973a5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -60,7 +60,10 @@ private object PostgresDialect extends JdbcDialect { case "bytea" => Some(BinaryType) case "timestamp" | "timestamptz" | "time" | "timetz" => Some(TimestampType) case "date" => Some(DateType) - case "numeric" | "decimal" => Some(DecimalType.bounded(precision, scale)) + case "numeric" | "decimal" if precision > 0 => Some(DecimalType.bounded(precision, scale)) + case "numeric" | "decimal" => + // SPARK-26538: handle numeric without explicit precision and scale. + Some(DecimalType. SYSTEM_DEFAULT) case _ => None } diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 7f301614523b..8d8decbdaca2 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -291,7 +291,7 @@ struct<> -- !query 31 output org.apache.spark.sql.catalyst.parser.ParseException -Exception parsing DATE(line 1, pos 7) +Cannot parse the DATE value: mar 11 2016(line 1, pos 7) == SQL == select date 'mar 11 2016' @@ -313,7 +313,7 @@ struct<> -- !query 33 output org.apache.spark.sql.catalyst.parser.ParseException -Timestamp format must be yyyy-mm-dd hh:mm:ss[.fffffffff](line 1, pos 7) +Cannot parse the TIMESTAMP value: 2016-33-11 20:54:00.000(line 1, pos 7) == SQL == select timestamp '2016-33-11 20:54:00.000' diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala index 68f7de047b39..69728efcd1d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala @@ -21,6 +21,7 @@ import com.esotericsoftware.kryo.{Kryo, Serializer} import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.SparkConf +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.sql.test.SharedSQLContext @@ -33,7 +34,7 @@ class DatasetSerializerRegistratorSuite extends QueryTest with SharedSQLContext override protected def sparkConf: SparkConf = { // Make sure we use the KryoRegistrator - super.sparkConf.set("spark.kryo.registrator", TestRegistrator().getClass.getCanonicalName) + super.sparkConf.set(KRYO_USER_REGISTRATORS, TestRegistrator().getClass.getCanonicalName) } test("Kryo registrator") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index c90b15814a53..ab8294838e75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.ScroogeLikeExample import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} +import org.apache.spark.sql.catalyst.plans.logical.SerializeFromObject import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} @@ -475,7 +476,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val keyValue1 = ds.groupByKey(t => (t._1, "key")).mapValues(t => (t._2, "value")) val agged1 = keyValue1.mapGroups { case (g, iter) => (g._1, iter.map(_._1).sum) } - checkDataset(agged, ("a", 30), ("b", 3), ("c", 1)) + checkDataset(agged1, ("a", 30), ("b", 3), ("c", 1)) } test("groupBy function, reduce") { @@ -1667,6 +1668,16 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val exceptDF = inputDF.filter(col("a").isin("0") or col("b") > "c") checkAnswer(inputDF.except(exceptDF), Seq(Row("1", null))) } + + test("SPARK-26619: Prune the unused serializers from SerializeFromObjec") { + val data = Seq(("a", 1), ("b", 2), ("c", 3)) + val ds = data.toDS().map(t => (t._1, t._2 + 1)).select("_1") + val serializer = ds.queryExecution.optimizedPlan.collect { + case s: SerializeFromObject => s + }.head + assert(serializer.serializer.size == 1) + checkAnswer(ds, Seq(Row("a"), Row("b"), Row("c"))) + } } case class TestDataUnion(x: Int, y: Int, z: Int) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index ce475922eb5e..ec688282d574 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -25,16 +25,25 @@ class ExplainSuite extends QueryTest with SharedSQLContext { import testImplicits._ /** - * Runs the plan and makes sure the plans contains all of the keywords. + * Get the explain from a DataFrame and run the specified action on it. */ - private def checkKeywordsExistsInExplain(df: DataFrame, keywords: String*): Unit = { + private def withNormalizedExplain(df: DataFrame, extended: Boolean)(f: String => Unit) = { val output = new java.io.ByteArrayOutputStream() Console.withOut(output) { - df.explain(extended = true) + df.explain(extended = extended) } val normalizedOutput = output.toString.replaceAll("#\\d+", "#x") - for (key <- keywords) { - assert(normalizedOutput.contains(key)) + f(normalizedOutput) + } + + /** + * Runs the plan and makes sure the plans contains all of the keywords. + */ + private def checkKeywordsExistsInExplain(df: DataFrame, keywords: String*): Unit = { + withNormalizedExplain(df, extended = true) { normalizedOutput => + for (key <- keywords) { + assert(normalizedOutput.contains(key)) + } } } @@ -182,6 +191,15 @@ class ExplainSuite extends QueryTest with SharedSQLContext { "id#xL AS nullif(`id`, 'x')#xL, coalesce(cast(id#xL as string), x) AS nvl(`id`, 'x')#x, " + "x AS nvl2(`id`, 'x', 'y')#x]") } + + test("SPARK-26659: explain of DataWritingCommandExec should not contain duplicate cmd.nodeName") { + withTable("temptable") { + val df = sql("create table temptable using parquet as select * from range(2)") + withNormalizedExplain(df, extended = false) { normalizedOutput => + assert("Create\\w*?TableAsSelectCommand".r.findAllMatchIn(normalizedOutput).length == 1) + } + } + } } case class ExplainSingleData(id: Int) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala index 55f210cb04db..30a3d54fd833 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext class JoinHintSuite extends PlanTest with SharedSQLContext { @@ -100,7 +101,7 @@ class JoinHintSuite extends PlanTest with SharedSQLContext { } } - test("hint preserved after join reorder") { + test("hints prevent join reorder") { withTempView("a", "b", "c") { df1.createOrReplaceTempView("a") df2.createOrReplaceTempView("b") @@ -118,12 +119,10 @@ class JoinHintSuite extends PlanTest with SharedSQLContext { verifyJoinHint( sql("select /*+ broadcast(a, c)*/ * from a, c, b " + "where a.a1 = b.b1 and b.b1 = c.c1"), - JoinHint( - None, - Some(HintInfo(broadcast = true))) :: + JoinHint.NONE :: JoinHint( Some(HintInfo(broadcast = true)), - None):: Nil + Some(HintInfo(broadcast = true))):: Nil ) verifyJoinHint( sql("select /*+ broadcast(b, c)*/ * from a, c, b " + @@ -199,4 +198,21 @@ class JoinHintSuite extends PlanTest with SharedSQLContext { None) :: Nil ) } + + test("hints prevent cost-based join reorder") { + withSQLConf(SQLConf.CBO_ENABLED.key -> "true", SQLConf.JOIN_REORDER_ENABLED.key -> "true") { + val join = df.join(df, "id") + val broadcasted = join.hint("broadcast") + verifyJoinHint( + join.join(broadcasted, "id").join(broadcasted, "id"), + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: + JoinHint( + None, + Some(HintInfo(broadcast = true))) :: + JoinHint.NONE :: JoinHint.NONE :: JoinHint.NONE :: Nil + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index cf25f1ce910d..d83deb17a090 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.columnar.InMemoryRelation @@ -234,7 +235,9 @@ object QueryTest { checkToRDD: Boolean = true): Option[String] = { val isSorted = df.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty if (checkToRDD) { - df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] + SQLExecution.withSQLConfPropagated(df.sparkSession) { + df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] + } } val sparkAnswer = try df.collect().toSeq catch { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 656da9fa0180..806f0b2239fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -240,6 +240,16 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Seq(Row("1"), Row("2"))) } + test("SPARK-11226 Skip empty line in json file") { + spark.read + .json(Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}", "").toDS()) + .createOrReplaceTempView("d") + + checkAnswer( + sql("select count(1) from d"), + Seq(Row(3))) + } + test("SPARK-8828 sum should return null if all input values are null") { checkAnswer( sql("select sum(a), avg(a) from allNulls"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index 44bf8624a6bc..10b17571d2aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.internal.SQLConf /** @@ -38,7 +39,7 @@ class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { test("create with config options and propagate them to SparkContext and SparkSession") { val session = SparkSession.builder() .master("local") - .config("spark.ui.enabled", value = false) + .config(UI_ENABLED.key, value = false) .config("some-config", "v2") .getOrCreate() assert(session.sparkContext.conf.get("some-config") == "v2") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 4a439940beb7..74f33f6c8139 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.scalatest.BeforeAndAfterAll import org.apache.spark.{MapOutputStatistics, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ import org.apache.spark.sql.execution.exchange.{ExchangeCoordinator, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.functions._ @@ -262,7 +263,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { new SparkConf(false) .setMaster("local[*]") .setAppName("test") - .set("spark.ui.enabled", "false") + .set(UI_ENABLED, false) .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala index bde2de5b39fd..211870f1c8fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala @@ -17,15 +17,18 @@ package org.apache.spark.sql.execution +import scala.concurrent.{Future, TimeoutException} +import scala.concurrent.duration._ import scala.util.Random -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, IdentityBroadcastMode, SinglePartition} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.util.ThreadUtils class ExchangeSuite extends SparkPlanTest with SharedSQLContext { import testImplicits._ @@ -132,4 +135,33 @@ class ExchangeSuite extends SparkPlanTest with SharedSQLContext { val projection2 = cached.select("_1", "_3").queryExecution.executedPlan assert(!projection1.sameResult(projection2)) } + + test("SPARK-26601: Make broadcast-exchange thread pool configurable") { + val previousNumber = SparkSession.getActiveSession.get.sparkContext.conf + .get(StaticSQLConf.BROADCAST_EXCHANGE_MAX_THREAD_THREASHOLD) + + SparkSession.getActiveSession.get.sparkContext.conf. + set(StaticSQLConf.BROADCAST_EXCHANGE_MAX_THREAD_THREASHOLD, 1) + assert(SQLConf.get.getConf(StaticSQLConf.BROADCAST_EXCHANGE_MAX_THREAD_THREASHOLD) === 1) + + Future { + Thread.sleep(5*1000) + } (BroadcastExchangeExec.executionContext) + + val f = Future {} (BroadcastExchangeExec.executionContext) + intercept[TimeoutException] { + ThreadUtils.awaitResult(f, 3 seconds) + } + + var executed = false + val ef = Future { + executed = true + } (BroadcastExchangeExec.executionContext) + ThreadUtils.awaitResult(ef, 3 seconds) + assert(executed) + + // for other test + SparkSession.getActiveSession.get.sparkContext.conf. + set(StaticSQLConf.BROADCAST_EXCHANGE_MAX_THREAD_THREASHOLD, previousNumber) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala index e174dc6f31a4..0869e25674e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala @@ -45,8 +45,8 @@ object ExternalAppendOnlyUnsafeRowArrayBenchmark extends BenchmarkBase { private val conf = new SparkConf(false) // Make the Java serializer write a reset instruction (TC_RESET) after each object to test // for a bug we had with bytes written past the last object in a batch (SPARK-2792) - .set("spark.serializer.objectStreamReset", "1") - .set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + .set(config.SERIALIZER_OBJECT_STREAM_RESET, 1) + .set(config.SERIALIZER, "org.apache.spark.serializer.JavaSerializer") private def withFakeTaskContext(f: => Unit): Unit = { val sc = new SparkContext("local", "test", conf) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala index 4205b3f79a97..bbce4705871d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -17,10 +17,27 @@ package org.apache.spark.sql.execution +import java.sql.{Date, Timestamp} + import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT} +import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} + +class HiveResultSuite extends SparkFunSuite with SharedSQLContext { + import testImplicits._ -class HiveResultSuite extends SparkFunSuite { + test("date formatting in hive result") { + val date = "2018-12-28" + val executedPlan = Seq(Date.valueOf(date)).toDS().queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan) + assert(result.head == date) + } + + test("timestamp formatting in hive result") { + val timestamp = "2018-12-28 01:02:03" + val executedPlan = Seq(Timestamp.valueOf(timestamp)).toDS().queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan) + assert(result.head == timestamp) + } test("toHiveString correctly handles UDTs") { val point = new ExamplePoint(50.0, 50.0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index 963e42517b44..1640a9611ec3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -21,6 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} import java.util.Properties import org.apache.spark._ +import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests.TEST_MEMORY import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.RDD @@ -98,9 +99,10 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkSession { test("SPARK-10466: external sorter spilling with unsafe row serializer") { val conf = new SparkConf() - .set("spark.shuffle.spill.initialMemoryThreshold", "1") - .set("spark.shuffle.sort.bypassMergeThreshold", "0") + .set(SHUFFLE_SPILL_INITIAL_MEM_THRESHOLD, 1L) + .set(SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD, 0) .set(TEST_MEMORY, 80000L) + spark = SparkSession.builder().master("local").appName("test").config(conf).getOrCreate() val outputFile = File.createTempFile("test-unsafe-row-serializer-spill", "") outputFile.deleteOnExit() @@ -127,7 +129,7 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkSession { } test("SPARK-10403: unsafe row serializer with SortShuffleManager") { - val conf = new SparkConf().set("spark.shuffle.manager", "sort") + val conf = new SparkConf().set(SHUFFLE_MANAGER, "sort") spark = SparkSession.builder().master("local").appName("test").config(conf).getOrCreate() val row = Row("Hello", 123) val unsafeRow = toUnsafeRow(row, Array(StringType, IntegerType)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala index aca7081c0d3d..bd2470ee2066 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala @@ -23,6 +23,7 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.SQLHelper @@ -50,7 +51,7 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { .set("spark.master", "local[1]") .setIfMissing("spark.driver.memory", "3g") .setIfMissing("spark.executor.memory", "3g") - .setIfMissing("spark.ui.enabled", "false") + .setIfMissing(UI_ENABLED, false) val spark = SparkSession.builder.config(conf).getOrCreate() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 017b74aabff7..b04024371713 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -23,6 +23,7 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.functions.monotonically_increasing_id @@ -48,7 +49,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { .set("spark.master", "local[1]") .setIfMissing("spark.driver.memory", "3g") .setIfMissing("spark.executor.memory", "3g") - .setIfMissing("spark.ui.enabled", "false") + .setIfMissing(UI_ENABLED, false) .setIfMissing("orc.compression", "snappy") .setIfMissing("spark.sql.parquet.compression.codec", "snappy") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala new file mode 100644 index 000000000000..cf4a34b20627 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions.{array, struct} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +/** + * A benchmark that compares the performance of different ways to evaluate SQL IN expressions. + * + * Specifically, this class compares the if-based approach, which might iterate through all items + * inside the IN value list, to other options with better worst-case time complexities (e.g., sets). + * + * To run this benchmark: + * {{{ + * 1. without sbt: bin/spark-submit --class + * 2. build/sbt "sql/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/InExpressionBenchmark-results.txt". + * }}} + */ +object InExpressionBenchmark extends SqlBasedBenchmark { + + import spark.implicits._ + + private def runByteBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems bytes" + val values = (Byte.MinValue until Byte.MinValue + numItems).map(v => s"${v}Y") + val df = spark.range(0, numRows).select($"id".cast(ByteType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runShortBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems shorts" + val values = (1 to numItems).map(v => s"${v}S") + val df = spark.range(0, numRows).select($"id".cast(ShortType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runIntBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems ints" + val values = 1 to numItems + val df = spark.range(0, numRows).select($"id".cast(IntegerType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runLongBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems longs" + val values = (1 to numItems).map(v => s"${v}L") + val df = spark.range(0, numRows).toDF("id") + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runFloatBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems floats" + val values = (1 to numItems).map(v => s"CAST($v AS float)") + val df = spark.range(0, numRows).select($"id".cast(FloatType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runDoubleBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems doubles" + val values = (1 to numItems).map(v => s"$v.0D") + val df = spark.range(0, numRows).select($"id".cast(DoubleType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runSmallDecimalBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems small decimals" + val values = (1 to numItems).map(v => s"CAST($v AS decimal(12, 1))") + val df = spark.range(0, numRows).select($"id".cast(DecimalType(12, 1))) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runLargeDecimalBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems large decimals" + val values = (1 to numItems).map(v => s"9223372036854775812.10539$v") + val df = spark.range(0, numRows).select($"id".cast(DecimalType(30, 7))) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runStringBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems strings" + val values = (1 to numItems).map(n => s"'$n'") + val df = spark.range(0, numRows).select($"id".cast(StringType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runTimestampBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems timestamps" + val values = (1 to numItems).map(m => s"CAST('1970-01-01 01:00:00.$m' AS timestamp)") + val df = spark.range(0, numRows).select($"id".cast(TimestampType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runDateBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems dates" + val values = (1 to numItems).map(n => 1970 + n).map(y => s"CAST('$y-01-01' AS date)") + val df = spark.range(0, numRows).select($"id".cast(TimestampType).cast(DateType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runArrayBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems arrays" + val values = (1 to numItems).map(i => s"array($i)") + val df = spark.range(0, numRows).select(array($"id").as("id")) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runStructBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems structs" + val values = (1 to numItems).map(i => s"struct($i)") + val df = spark.range(0, numRows).select(struct($"id".as("col1")).as("id")) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runBenchmark( + name: String, + df: DataFrame, + values: Seq[Any], + numRows: Long, + minNumIters: Int): Unit = { + + val benchmark = new Benchmark(name, numRows, minNumIters, output = output) + + df.createOrReplaceTempView("t") + + def testClosure(): Unit = { + val df = spark.sql(s"SELECT * FROM t WHERE id IN (${values.mkString(",")})") + df.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark.addCase("In expression") { _ => + withSQLConf(SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> values.size.toString) { + testClosure() + } + } + + benchmark.addCase("InSet expression") { _ => + withSQLConf(SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "1") { + testClosure() + } + } + + benchmark.run() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val numItemsSeq = Seq(5, 10, 25, 50, 100, 200) + val largeNumRows = 10000000 + val smallNumRows = 1000000 + val minNumIters = 5 + + runBenchmark("In Expression Benchmark") { + numItemsSeq.foreach { numItems => + runByteBenchmark(numItems, largeNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runShortBenchmark(numItems, largeNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runIntBenchmark(numItems, largeNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runLongBenchmark(numItems, largeNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runFloatBenchmark(numItems, largeNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runDoubleBenchmark(numItems, largeNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runSmallDecimalBenchmark(numItems, smallNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runLargeDecimalBenchmark(numItems, smallNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runStringBenchmark(numItems, smallNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runTimestampBenchmark(numItems, largeNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runDateBenchmark(numItems, largeNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runArrayBenchmark(numItems, smallNumRows, minNumIters) + } + numItemsSeq.foreach { numItems => + runStructBenchmark(numItems, smallNumRows, minNumIters) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 8f575a371c98..49dd9c22e831 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -1126,7 +1126,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Row(null, null, null), Row(null, null, null), Row(null, null, null), - Row(null, null, null), Row("str_a_4", "str_b_4", "str_c_4"), Row(null, null, null)) ) @@ -1148,7 +1147,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer( jsonDF.select($"a", $"b", $"c", $"_unparsed"), Row(null, null, null, "{") :: - Row(null, null, null, "") :: Row(null, null, null, """{"a":1, b:2}""") :: Row(null, null, null, """{"a":{, b:3}""") :: Row("str_a_4", "str_b_4", "str_c_4", null) :: @@ -1163,7 +1161,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer( jsonDF.filter($"_unparsed".isNotNull).select($"_unparsed"), Row("{") :: - Row("") :: Row("""{"a":1, b:2}""") :: Row("""{"a":{, b:3}""") :: Row("]") :: Nil @@ -1185,7 +1182,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkAnswer( jsonDF.selectExpr("a", "b", "c", "_malformed"), Row(null, null, null, "{") :: - Row(null, null, null, "") :: Row(null, null, null, """{"a":1, b:2}""") :: Row(null, null, null, """{"a":{, b:3}""") :: Row("str_a_4", "str_b_4", "str_c_4", null) :: @@ -1451,109 +1447,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { }) } - test("backward compatibility") { - withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> "true") { - // This test we make sure our JSON support can read JSON data generated by previous version - // of Spark generated through toJSON method and JSON data source. - // The data is generated by the following program. - // Here are a few notes: - // - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13) - // in the JSON object. - // - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to - // JSON objects generated by those Spark versions (col17). - // - If the type is NullType, we do not write data out. - - // Create the schema. - val struct = - StructType( - StructField("f1", FloatType, true) :: - StructField("f2", ArrayType(BooleanType), true) :: Nil) - - val dataTypes = - Seq( - StringType, BinaryType, NullType, BooleanType, - ByteType, ShortType, IntegerType, LongType, - FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), - DateType, TimestampType, - ArrayType(IntegerType), MapType(StringType, LongType), struct, - new TestUDT.MyDenseVectorUDT()) - val fields = dataTypes.zipWithIndex.map { case (dataType, index) => - StructField(s"col$index", dataType, nullable = true) - } - val schema = StructType(fields) - - val constantValues = - Seq( - "a string in binary".getBytes(StandardCharsets.UTF_8), - null, - true, - 1.toByte, - 2.toShort, - 3, - Long.MaxValue, - 0.25.toFloat, - 0.75, - new java.math.BigDecimal(s"1234.23456"), - new java.math.BigDecimal(s"1.23456"), - java.sql.Date.valueOf("2015-01-01"), - java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"), - Seq(2, 3, 4), - Map("a string" -> 2000L), - Row(4.75.toFloat, Seq(false, true)), - new TestUDT.MyDenseVector(Array(0.25, 2.25, 4.25))) - val data = - Row.fromSeq(Seq("Spark " + spark.sparkContext.version) ++ constantValues) :: Nil - - // Data generated by previous versions. - // scalastyle:off - val existingJSONData = - """{"col0":"Spark 1.2.2","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"16436","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: Nil - // scalastyle:on - - // Generate data for the current version. - val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1), schema) - withTempPath { path => - df.write.format("json").mode("overwrite").save(path.getCanonicalPath) - - // df.toJSON will convert internal rows to external rows first and then generate - // JSON objects. While, df.write.format("json") will write internal rows directly. - val allJSON = - existingJSONData ++ - df.toJSON.collect() ++ - sparkContext.textFile(path.getCanonicalPath).collect() - - Utils.deleteRecursively(path) - sparkContext.parallelize(allJSON, 1).saveAsTextFile(path.getCanonicalPath) - - // Read data back with the schema specified. - val col0Values = - Seq( - "Spark 1.2.2", - "Spark 1.3.1", - "Spark 1.3.1", - "Spark 1.4.1", - "Spark 1.4.1", - "Spark 1.5.0", - "Spark 1.5.0", - "Spark " + spark.sparkContext.version, - "Spark " + spark.sparkContext.version) - val expectedResult = col0Values.map { v => - Row.fromSeq(Seq(v) ++ constantValues) - } - checkAnswer( - spark.read.format("json").schema(schema).load(path.getCanonicalPath), - expectedResult - ) - } - } - } - test("SPARK-11544 test pathfilter") { withTempPath { dir => val path = dir.getCanonicalPath @@ -1830,7 +1723,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val path = dir.getCanonicalPath primitiveFieldAndType .toDF("value") - .repartition(1) .write .option("compression", "GzIp") .text(path) @@ -2531,7 +2423,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } checkCount(2) - countForMalformedJSON(1, Seq("")) + countForMalformedJSON(0, Seq("")) } test("SPARK-25040: empty strings should be disallowed") { @@ -2592,53 +2484,45 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("inferring timestamp type") { - Seq(true, false).foreach { legacyParser => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { - def schemaOf(jsons: String*): StructType = spark.read.json(jsons.toDS).schema - - assert(schemaOf( - """{"a":"2018-12-17T10:11:12.123-01:00"}""", - """{"a":"2018-12-16T22:23:24.123-02:00"}""") === fromDDL("a timestamp")) - - assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":1}""") - === fromDDL("a string")) - assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":"123"}""") - === fromDDL("a string")) - - assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":null}""") - === fromDDL("a timestamp")) - assert(schemaOf("""{"a":null}""", """{"a":"2018-12-17T10:11:12.123-01:00"}""") - === fromDDL("a timestamp")) - } - } + def schemaOf(jsons: String*): StructType = spark.read.json(jsons.toDS).schema + + assert(schemaOf( + """{"a":"2018-12-17T10:11:12.123-01:00"}""", + """{"a":"2018-12-16T22:23:24.123-02:00"}""") === fromDDL("a timestamp")) + + assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":1}""") + === fromDDL("a string")) + assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":"123"}""") + === fromDDL("a string")) + + assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":null}""") + === fromDDL("a timestamp")) + assert(schemaOf("""{"a":null}""", """{"a":"2018-12-17T10:11:12.123-01:00"}""") + === fromDDL("a timestamp")) } test("roundtrip for timestamp type inferring") { - Seq(true, false).foreach { legacyParser => - withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { - val customSchema = new StructType().add("date", TimestampType) - withTempDir { dir => - val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.json" - val timestampsWithFormat = spark.read - .option("timestampFormat", "dd/MM/yyyy HH:mm") - .json(datesRecords) - assert(timestampsWithFormat.schema === customSchema) - - timestampsWithFormat.write - .format("json") - .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") - .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") - .save(timestampsWithFormatPath) - - val readBack = spark.read - .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") - .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") - .json(timestampsWithFormatPath) - - assert(readBack.schema === customSchema) - checkAnswer(readBack, timestampsWithFormat) - } - } + val customSchema = new StructType().add("date", TimestampType) + withTempDir { dir => + val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.json" + val timestampsWithFormat = spark.read + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .json(datesRecords) + assert(timestampsWithFormat.schema === customSchema) + + timestampsWithFormat.write + .format("json") + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .save(timestampsWithFormatPath) + + val readBack = spark.read + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .json(timestampsWithFormatPath) + + assert(readBack.schema === customSchema) + checkAnswer(readBack, timestampsWithFormat) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala new file mode 100644 index 000000000000..59de28688ec1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.noop + +import org.apache.spark.sql.test.SharedSQLContext + +class NoopSuite extends SharedSQLContext { + import testImplicits._ + + test("materialisation of all rows") { + val numElems = 10 + val accum = spark.sparkContext.longAccumulator + spark.range(numElems) + .map { x => + accum.add(1) + x + } + .write + .format("noop") + .save() + assert(accum.value == numElems) + } + + test("read partitioned data") { + val numElems = 100 + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(numElems) + .select('id mod 10 as "key", 'id as "value") + .write + .partitionBy("key") + .parquet(path) + + val accum = spark.sparkContext.longAccumulator + spark.read + .parquet(path) + .as[(Long, Long)] + .map { x => + accum.add(1) + x + } + .write.format("noop").save() + assert(accum.value == numElems) + } + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index ee12f3089243..cccd8e9ee8bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -24,11 +24,15 @@ import scala.collection.JavaConverters._ import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} -import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.{AnalysisException, Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -41,7 +45,7 @@ import org.apache.spark.sql.types._ */ class OrcFilterSuite extends OrcTest with SharedSQLContext { - private def checkFilterPredicate( + protected def checkFilterPredicate( df: DataFrame, predicate: Predicate, checker: (SearchArgument) => Unit): Unit = { @@ -50,24 +54,24 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { .select(output.map(e => Column(e)): _*) .where(Column(predicate)) - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - filters - }.flatten.reduceLeftOption(_ && _) - assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") - - val (_, selectedFilters, _) = - DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) - assert(selectedFilters.nonEmpty, "No filter is pushed down") - - val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) - assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") - checker(maybeFilter.get) + query.queryExecution.optimizedPlan match { + case PhysicalOperation(_, filters, + DataSourceV2Relation(orcTable: OrcTable, _, options)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = orcTable.newScanBuilder(new DataSourceOptions(options.asJava)) + scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) + val pushedFilters = scanBuilder.pushedFilters() + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pushedFilters") + checker(maybeFilter.get) + + case _ => + throw new AnalysisException("Can not match OrcTable in the query.") + } } - private def checkFilterPredicate + protected def checkFilterPredicate (predicate: Predicate, filterOperator: PredicateLeaf.Operator) (implicit df: DataFrame): Unit = { def checkComparisonOperator(filter: SearchArgument) = { @@ -77,7 +81,7 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { checkFilterPredicate(df, predicate, checkComparisonOperator) } - private def checkFilterPredicate + protected def checkFilterPredicate (predicate: Predicate, stringExpr: String) (implicit df: DataFrame): Unit = { def checkLogicalOperator(filter: SearchArgument) = { @@ -86,28 +90,32 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { checkFilterPredicate(df, predicate, checkLogicalOperator) } - private def checkNoFilterPredicate - (predicate: Predicate) + protected def checkNoFilterPredicate + (predicate: Predicate, noneSupported: Boolean = false) (implicit df: DataFrame): Unit = { val output = predicate.collect { case a: Attribute => a }.distinct val query = df .select(output.map(e => Column(e)): _*) .where(Column(predicate)) - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - filters - }.flatten.reduceLeftOption(_ && _) - assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") - - val (_, selectedFilters, _) = - DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) - assert(selectedFilters.nonEmpty, "No filter is pushed down") - - val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) - assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") + query.queryExecution.optimizedPlan match { + case PhysicalOperation(_, filters, + DataSourceV2Relation(orcTable: OrcTable, _, options)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = orcTable.newScanBuilder(new DataSourceOptions(options.asJava)) + scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) + val pushedFilters = scanBuilder.pushedFilters() + if (noneSupported) { + assert(pushedFilters.isEmpty, "Unsupported filters should not show in pushed filters") + } else { + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) + assert(maybeFilter.isEmpty, s"Couldn't generate filter predicate for $pushedFilters") + } + + case _ => + throw new AnalysisException("Can not match OrcTable in the query.") + } } test("filter pushdown - integer") { @@ -346,15 +354,15 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { } // ArrayType withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate('_1.isNull) + checkNoFilterPredicate('_1.isNull, noneSupported = true) } // BinaryType withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate('_1 <=> 1.b) + checkNoFilterPredicate('_1 <=> 1.b, noneSupported = true) } // MapType withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate('_1.isNotNull) + checkNoFilterPredicate('_1.isNotNull, noneSupported = true) } } @@ -419,3 +427,4 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { } } } + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala index d1911ea7f32a..4a695ac74c47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql.execution.datasources.orc import java.io.File +import org.apache.spark.SparkConf import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext // The data where the partitioning key exists only in the directory structure. @@ -227,3 +229,8 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { } class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext + +class OrcV1PartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext { + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 918dbcdfa1cc..d0b386b88c59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -31,7 +31,7 @@ import org.apache.orc.OrcConf.COMPRESS import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce.OrcInputFormat -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, RecordReaderIterator} @@ -574,7 +574,7 @@ abstract class OrcQueryTest extends OrcTest { val m1 = intercept[AnalysisException] { testAllCorruptFiles() }.getMessage - assert(m1.contains("Unable to infer schema for ORC")) + assert(m1.contains("Unable to infer schema")) testAllCorruptFilesWithoutSchemaInfer() } @@ -681,3 +681,8 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { } } } + +class OrcV1QuerySuite extends OrcQuerySuite { + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala new file mode 100644 index 000000000000..cf5bbb3fff70 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import scala.collection.JavaConverters._ + +import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.internal.SQLConf + +class OrcV1FilterSuite extends OrcFilterSuite { + + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") + + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + checker: (SearchArgument) => Unit): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + filters + }.flatten.reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + assert(selectedFilters.nonEmpty, "No filter is pushed down") + + val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") + checker(maybeFilter.get) + } + + override def checkFilterPredicate + (predicate: Predicate, filterOperator: PredicateLeaf.Operator) + (implicit df: DataFrame): Unit = { + def checkComparisonOperator(filter: SearchArgument) = { + val operator = filter.getLeaves.asScala + assert(operator.map(_.getOperator).contains(filterOperator)) + } + checkFilterPredicate(df, predicate, checkComparisonOperator) + } + + override def checkFilterPredicate + (predicate: Predicate, stringExpr: String) + (implicit df: DataFrame): Unit = { + def checkLogicalOperator(filter: SearchArgument) = { + assert(filter.toString == stringExpr) + } + checkFilterPredicate(df, predicate, checkLogicalOperator) + } + + override def checkNoFilterPredicate + (predicate: Predicate, noneSupported: Boolean = false) + (implicit df: DataFrame): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + filters + }.flatten.reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + assert(selectedFilters.nonEmpty, "No filter is pushed down") + + val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) + assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 88067358667c..864c1e99fbfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.{PartitionPath => Partition} import org.apache.spark.sql.execution.streaming.MemoryStream @@ -56,6 +56,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha val timeZone = TimeZone.getDefault() val timeZoneId = timeZone.getID + val df = DateFormatter() + val tf = TimestampFormatter(timestampPartitionPattern, timeZone) protected override def beforeAll(): Unit = { super.beforeAll() @@ -69,7 +71,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("column type inference") { def check(raw: String, literal: Literal, timeZone: TimeZone = timeZone): Unit = { - assert(inferPartitionColumnValue(raw, true, timeZone) === literal) + assert(inferPartitionColumnValue(raw, true, timeZone, df, tf) === literal) } check("10", Literal.create(10, IntegerType)) @@ -197,13 +199,13 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partition") { def check(path: String, expected: Option[PartitionValues]): Unit = { val actual = parsePartition(new Path(path), true, Set.empty[Path], - Map.empty, true, timeZone)._1 + Map.empty, true, timeZone, df, tf)._1 assert(expected === actual) } def checkThrows[T <: Throwable: Manifest](path: String, expected: String): Unit = { val message = intercept[T] { - parsePartition(new Path(path), true, Set.empty[Path], Map.empty, true, timeZone) + parsePartition(new Path(path), true, Set.empty[Path], Map.empty, true, timeZone, df, tf) }.getMessage assert(message.contains(expected)) @@ -249,7 +251,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha basePaths = Set(new Path("file://path/a=10")), Map.empty, true, - timeZone = timeZone)._1 + timeZone = timeZone, + df, + tf)._1 assert(partitionSpec1.isEmpty) @@ -260,7 +264,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha basePaths = Set(new Path("file://path")), Map.empty, true, - timeZone = timeZone)._1 + timeZone = timeZone, + df, + tf)._1 assert(partitionSpec2 == Option(PartitionValues( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 54c77dddc352..ce1dc6e159c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File -import java.sql.Timestamp import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.parquet.hadoop.ParquetOutputFormat @@ -187,12 +186,12 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext sql("insert into ts values (1, '2016-01-01 10:11:12.123456')") sql("insert into ts values (2, null)") sql("insert into ts values (3, '1965-01-01 10:11:12.123456')") - checkAnswer( - sql("select * from ts"), - Seq( - Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123456")), - Row(2, null), - Row(3, Timestamp.valueOf("1965-01-01 10:11:12.123456")))) + val expected = Seq( + (1, "2016-01-01 10:11:12.123456"), + (2, null), + (3, "1965-01-01 10:11:12.123456")) + .toDS().select('_1, $"_2".cast("timestamp")) + checkAnswer(sql("select * from ts"), expected) } // The microsecond portion is truncated when written as TIMESTAMP_MILLIS. @@ -206,30 +205,30 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext sql("insert into ts values (5, '1965-01-01 10:11:12.1')") sql("insert into ts values (6, '1965-01-01 10:11:12.123456789')") sql("insert into ts values (7, '0001-01-01 00:00:00.000000')") - checkAnswer( - sql("select * from ts"), - Seq( - Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123")), - Row(2, null), - Row(3, Timestamp.valueOf("1965-01-01 10:11:12.125")), - Row(4, Timestamp.valueOf("1965-01-01 10:11:12.125")), - Row(5, Timestamp.valueOf("1965-01-01 10:11:12.1")), - Row(6, Timestamp.valueOf("1965-01-01 10:11:12.123")), - Row(7, Timestamp.valueOf("0001-01-01 00:00:00.000")))) + val expected = Seq( + (1, "2016-01-01 10:11:12.123"), + (2, null), + (3, "1965-01-01 10:11:12.125"), + (4, "1965-01-01 10:11:12.125"), + (5, "1965-01-01 10:11:12.1"), + (6, "1965-01-01 10:11:12.123"), + (7, "0001-01-01 00:00:00.000")) + .toDS().select('_1, $"_2".cast("timestamp")) + checkAnswer(sql("select * from ts"), expected) // Read timestamps that were encoded as TIMESTAMP_MILLIS annotated as INT64 // with PARQUET_INT64_AS_TIMESTAMP_MILLIS set to false. withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "false") { - checkAnswer( - sql("select * from ts"), - Seq( - Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123")), - Row(2, null), - Row(3, Timestamp.valueOf("1965-01-01 10:11:12.125")), - Row(4, Timestamp.valueOf("1965-01-01 10:11:12.125")), - Row(5, Timestamp.valueOf("1965-01-01 10:11:12.1")), - Row(6, Timestamp.valueOf("1965-01-01 10:11:12.123")), - Row(7, Timestamp.valueOf("0001-01-01 00:00:00.000")))) + val expected = Seq( + (1, "2016-01-01 10:11:12.123"), + (2, null), + (3, "1965-01-01 10:11:12.125"), + (4, "1965-01-01 10:11:12.125"), + (5, "1965-01-01 10:11:12.1"), + (6, "1965-01-01 10:11:12.123"), + (7, "0001-01-01 00:00:00.000")) + .toDS().select('_1, $"_2".cast("timestamp")) + checkAnswer(sql("select * from ts"), expected) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 9a02529a2550..4d15f38321a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.SchemaPruningTest import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StructType @@ -217,6 +218,41 @@ class ParquetSchemaPruningSuite Row("Y.") :: Nil) } + testSchemaPruning("select one complex field and having is null predicate on another " + + "complex field") { + val query = sql("select * from contacts") + .where("name.middle is not null") + .select( + "id", + "name.first", + "name.middle", + "name.last" + ) + .where("last = 'Jones'") + .select(count("id")).toDF() + checkScan(query, + "struct>") + checkAnswer(query, Row(0) :: Nil) + } + + testSchemaPruning("select one deep nested complex field and having is null predicate on " + + "another deep nested complex field") { + val query = sql("select * from contacts") + .where("employer.company.address is not null") + .selectExpr( + "id", + "name.first", + "name.middle", + "name.last", + "employer.id as employer_id" + ) + .where("employer_id = 0") + .select(count("id")).toDF() + checkScan(query, + "struct>>") + checkAnswer(query, Row(1) :: Nil) + } + private def testSchemaPruning(testName: String)(testThunk: => Unit) { test(s"Spark vectorized reader - without partition data column - $testName") { withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index 7b55e839e3b4..1c8991010504 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -22,7 +22,8 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, import scala.util.Random import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.catalyst.InternalRow @@ -309,7 +310,7 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { test("Spark-14521") { val ser = new KryoSerializer( - (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() + (new SparkConf).set(KRYO_REFERENCE_TRACKING, false)).newInstance() val key = Seq(BoundReference(0, LongType, false)) // Testing Kryo serialization of HashedRelation diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 6174ec4c8908..98a8ad5eeb2b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -96,9 +96,9 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df = testData2.groupBy().count() // 2 partitions val expected1 = Seq( Map("number of output rows" -> 2L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"), + "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)"), Map("number of output rows" -> 1L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) + "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)")) val shuffleExpected1 = Map( "records read" -> 2L, "local blocks read" -> 2L, @@ -114,9 +114,9 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df2 = testData2.groupBy('a).count() val expected2 = Seq( Map("number of output rows" -> 4L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"), + "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)"), Map("number of output rows" -> 3L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) + "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)")) val shuffleExpected2 = Map( "records read" -> 4L, "local blocks read" -> 4L, @@ -162,7 +162,7 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") + val probes = metrics(nodeId)._2("avg hash probe bucket list iters (min, med, max)") probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => assert(probe.toDouble > 1.0) } @@ -570,7 +570,7 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared testSparkPlanMetrics(df, 1, Map( 0L -> (("Scan parquet default.testdataforscan", Map( "number of output rows" -> 3L, - "number of files" -> 2L)))) + "number of files read" -> 2L)))) ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala index 0e13f7dd55ba..f12eeaa58064 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -84,8 +84,10 @@ trait SQLMetricsTestUtils extends SQLTestUtils { assert(metricValue == expected) } - val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get - val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt + val totalNumBytesMetric = executedNode.metrics.find( + _.name == "written output total (min, med, max)").get + val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "") + .split(" ").head.trim.toDouble assert(totalNumBytes > 0) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index 9268306ce427..0e36e7f5da12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -178,5 +178,11 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), None, Some(5L))) intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), Some(5L))) intercept[IllegalStateException](verifyBatchIds(Seq(1, 2, 4, 5), Some(1L), Some(5L))) + + // Related to SPARK-26629, this capatures the behavior for verifyBatchIds when startId > endId + intercept[IllegalStateException](verifyBatchIds(Seq(), Some(2L), Some(1L))) + intercept[AssertionError](verifyBatchIds(Seq(2), Some(2L), Some(1L))) + intercept[AssertionError](verifyBatchIds(Seq(1), Some(2L), Some(1L))) + intercept[AssertionError](verifyBatchIds(Seq(0), Some(2L), Some(1L))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index aefa5da94481..284900b68ae5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -850,8 +850,11 @@ class JDBCSuite extends QueryTest test("PostgresDialect type mapping") { val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") + val md = new MetadataBuilder().putLong("scale", 0) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "json", 1, null) === Some(StringType)) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "jsonb", 1, null) === Some(StringType)) + assert(Postgres.getCatalystType(java.sql.Types.ARRAY, "_numeric", 0, md) == + Some(ArrayType(DecimalType.SYSTEM_DEFAULT))) assert(Postgres.getJDBCType(FloatType).map(_.databaseTypeDefinition).get == "FLOAT4") assert(Postgres.getJDBCType(DoubleType).map(_.databaseTypeDefinition).get == "FLOAT8") val errMsg = intercept[IllegalArgumentException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 0b6d93975dae..4f1ae069d4b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -170,7 +170,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to more part files. val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 10) - spark.read.json(rdd1.toDS()).createOrReplaceTempView("jt2") + spark.read.json(rdd2.toDS()).createOrReplaceTempView("jt2") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala index d282193d35d7..c60ea4a2f9f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala @@ -329,8 +329,8 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { .format(classOf[DataSourceV2WithSessionConfig].getName).load() val options = df.queryExecution.optimizedPlan.collectFirst { case d: DataSourceV2Relation => d.options - } - assert(options.get.get(optionName) == Some("false")) + }.get + assert(options.get(optionName).get == "false") } } @@ -356,13 +356,11 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { val cls = classOf[SimpleWriteOnlyDataSource] val path = file.getCanonicalPath val df = spark.range(5).select('id as 'i, -'id as 'j) - try { - df.write.format(cls.getName).option("path", path).mode("error").save() - df.write.format(cls.getName).option("path", path).mode("overwrite").save() - df.write.format(cls.getName).option("path", path).mode("ignore").save() - } catch { - case e: SchemaReadAttemptException => fail("Schema read was attempted.", e) - } + // non-append mode should not throw exception, as they don't access schema. + df.write.format(cls.getName).option("path", path).mode("error").save() + df.write.format(cls.getName).option("path", path).mode("overwrite").save() + df.write.format(cls.getName).option("path", path).mode("ignore").save() + // append mode will access schema and should throw exception. intercept[SchemaReadAttemptException] { df.write.format(cls.getName).option("path", path).mode("append").save() } @@ -680,10 +678,12 @@ object SpecificReaderFactory extends PartitionReaderFactory { class SchemaReadAttemptException(m: String) extends RuntimeException(m) class SimpleWriteOnlyDataSource extends SimpleWritableDataSource { - override def writeSchema(): StructType = { - // This is a bit hacky since this source implements read support but throws - // during schema retrieval. Might have to rewrite but it's done - // such so for minimised changes. - throw new SchemaReadAttemptException("read is not supported") + + override def getTable(options: DataSourceOptions): Table = { + new MyTable(options) { + override def schema(): StructType = { + throw new SchemaReadAttemptException("schema should not be read.") + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala new file mode 100644 index 000000000000..f57c581fd800 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources.v2 + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetTest} +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.ScanBuilder +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType + +class DummyReadOnlyFileDataSourceV2 extends FileDataSourceV2 { + + override def fallBackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] + + override def shortName(): String = "parquet" + + override def getTable(options: DataSourceOptions): Table = { + new DummyReadOnlyFileTable + } +} + +class DummyReadOnlyFileTable extends Table with SupportsBatchRead { + override def name(): String = "dummy" + + override def schema(): StructType = StructType(Nil) + + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { + throw new AnalysisException("Dummy file reader") + } +} + +class FileDataSourceV2FallBackSuite extends QueryTest with ParquetTest with SharedSQLContext { + + private val dummyParquetReaderV2 = classOf[DummyReadOnlyFileDataSourceV2].getName + + test("Fall back to v1 when writing to file with read only FileDataSourceV2") { + val df = spark.range(10).toDF() + withTempPath { file => + val path = file.getCanonicalPath + // Writing file should fall back to v1 and succeed. + df.write.format(dummyParquetReaderV2).save(path) + + // Validate write result with [[ParquetFileFormat]]. + checkAnswer(spark.read.parquet(path), df) + + // Dummy File reader should fail as expected. + val exception = intercept[AnalysisException] { + spark.read.format(dummyParquetReaderV2).load(path).collect() + } + assert(exception.message.equals("Dummy file reader")) + } + } + + test("Fall back read path to v1 with configuration USE_V1_SOURCE_READER_LIST") { + val df = spark.range(10).toDF() + withTempPath { file => + val path = file.getCanonicalPath + df.write.parquet(path) + Seq( + "foo,parquet,bar", + "ParQuet,bar,foo", + s"foobar,$dummyParquetReaderV2" + ).foreach { fallbackReaders => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> fallbackReaders) { + // Reading file should fall back to v1 and succeed. + checkAnswer(spark.read.format(dummyParquetReaderV2).load(path), df) + checkAnswer(sql(s"SELECT * FROM parquet.`$path`"), df) + } + } + + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "foo,bar") { + // Dummy File reader should fail as DISABLED_V2_FILE_DATA_SOURCE_READERS doesn't include it. + val exception = intercept[AnalysisException] { + spark.read.format(dummyParquetReaderV2).load(path).collect() + } + assert(exception.message.equals("Dummy file reader")) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala index 82bb4fa33a3a..6e4f2bbcd6b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.sources.v2 import java.io.{BufferedReader, InputStreamReader, IOException} -import java.util.Optional import scala.collection.JavaConverters._ @@ -35,15 +34,13 @@ import org.apache.spark.util.SerializableConfiguration /** * A HDFS based transactional writable data source. - * Each task writes data to `target/_temporary/queryId/$jobId-$partitionId-$attemptNumber`. - * Each job moves files from `target/_temporary/queryId/` to `target`. + * Each task writes data to `target/_temporary/uniqueId/$jobId-$partitionId-$attemptNumber`. + * Each job moves files from `target/_temporary/uniqueId/` to `target`. */ class SimpleWritableDataSource extends DataSourceV2 - with TableProvider - with BatchWriteSupportProvider - with SessionConfigSupport { + with TableProvider with SessionConfigSupport { - protected def writeSchema(): StructType = new StructType().add("i", "long").add("j", "long") + private val tableSchema = new StructType().add("i", "long").add("j", "long") override def keyPrefix: String = "simpleWritableDataSource" @@ -68,22 +65,50 @@ class SimpleWritableDataSource extends DataSourceV2 new CSVReaderFactory(serializableConf) } - override def readSchema(): StructType = writeSchema + override def readSchema(): StructType = tableSchema } - override def getTable(options: DataSourceOptions): Table = { - val path = new Path(options.get("path").get()) - val conf = SparkContext.getActive.get.hadoopConfiguration - new SimpleBatchTable { - override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { - new MyScanBuilder(path.toUri.toString, conf) + class MyWriteBuilder(path: String) extends WriteBuilder with SupportsSaveMode { + private var queryId: String = _ + private var mode: SaveMode = _ + + override def withQueryId(queryId: String): WriteBuilder = { + this.queryId = queryId + this + } + + override def mode(mode: SaveMode): WriteBuilder = { + this.mode = mode + this + } + + override def buildForBatch(): BatchWrite = { + assert(mode != null) + + val hadoopPath = new Path(path) + val hadoopConf = SparkContext.getActive.get.hadoopConfiguration + val fs = hadoopPath.getFileSystem(hadoopConf) + + if (mode == SaveMode.ErrorIfExists) { + if (fs.exists(hadoopPath)) { + throw new RuntimeException("data already exists.") + } + } + if (mode == SaveMode.Ignore) { + if (fs.exists(hadoopPath)) { + return null + } + } + if (mode == SaveMode.Overwrite) { + fs.delete(hadoopPath, true) } - override def schema(): StructType = writeSchema + val pathStr = hadoopPath.toUri.toString + new MyBatchWrite(queryId, pathStr, hadoopConf) } } - class WritSupport(queryId: String, path: String, conf: Configuration) extends BatchWriteSupport { + class MyBatchWrite(queryId: String, path: String, conf: Configuration) extends BatchWrite { override def createBatchWriterFactory(): DataWriterFactory = { SimpleCounter.resetCounter new CSVDataWriterFactory(path, queryId, new SerializableConfiguration(conf)) @@ -116,33 +141,23 @@ class SimpleWritableDataSource extends DataSourceV2 } } - override def createBatchWriteSupport( - queryId: String, - schema: StructType, - mode: SaveMode, - options: DataSourceOptions): Optional[BatchWriteSupport] = { - assert(!SparkContext.getActive.get.conf.getBoolean("spark.speculation", false)) + class MyTable(options: DataSourceOptions) extends SimpleBatchTable with SupportsBatchWrite { + private val path = options.get("path").get() + private val conf = SparkContext.getActive.get.hadoopConfiguration - val path = new Path(options.get("path").get()) - val conf = SparkContext.getActive.get.hadoopConfiguration - val fs = path.getFileSystem(conf) + override def schema(): StructType = tableSchema - if (mode == SaveMode.ErrorIfExists) { - if (fs.exists(path)) { - throw new RuntimeException("data already exists.") - } + override def newScanBuilder(options: DataSourceOptions): ScanBuilder = { + new MyScanBuilder(new Path(path).toUri.toString, conf) } - if (mode == SaveMode.Ignore) { - if (fs.exists(path)) { - return Optional.empty() - } - } - if (mode == SaveMode.Overwrite) { - fs.delete(path, true) + + override def newWriteBuilder(options: DataSourceOptions): WriteBuilder = { + new MyWriteBuilder(path) } + } - val pathStr = path.toUri.toString - Optional.of(new WritSupport(queryId, pathStr, conf)) + override def getTable(options: DataSourceOptions): Table = { + new MyTable(options) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index de664cafed3b..9235c6d7c896 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -48,21 +48,33 @@ abstract class FileStreamSourceTest * `FileStreamSource` actually being used in the execution. */ abstract class AddFileData extends AddData { + private val _qualifiedBasePath = PrivateMethod[Path]('qualifiedBasePath) + + private def isSamePath(fileSource: FileStreamSource, srcPath: File): Boolean = { + val path = (fileSource invokePrivate _qualifiedBasePath()).toString.stripPrefix("file:") + path == srcPath.getCanonicalPath + } + override def addData(query: Option[StreamExecution]): (Source, Offset) = { require( query.nonEmpty, "Cannot add data when there is no query for finding the active file stream source") val sources = getSourcesFromStreamingQuery(query.get) - if (sources.isEmpty) { + val source = if (sources.isEmpty) { throw new Exception( "Could not find file source in the StreamExecution logical plan to add data to") - } else if (sources.size > 1) { - throw new Exception( - "Could not select the file source in the StreamExecution logical plan as there" + - "are multiple file sources:\n\t" + sources.mkString("\n\t")) + } else if (sources.size == 1) { + sources.head + } else { + val matchedSources = sources.filter(isSamePath(_, src)) + if (matchedSources.size != 1) { + throw new Exception( + "Could not select the file source in StreamExecution as there are multiple" + + s" file sources and none / more than one matches $src:\n" + sources.mkString("\n")) + } + matchedSources.head } - val source = sources.head val newOffset = source.withBatchingLocked { addData(source) new FileStreamSourceOffset(source.currentLogOffset + 1) @@ -71,6 +83,9 @@ abstract class FileStreamSourceTest (source, newOffset) } + /** Source directory to add file data to */ + protected def src: File + protected def addData(source: FileStreamSource): Unit } @@ -1494,6 +1509,54 @@ class FileStreamSourceSuite extends FileStreamSourceTest { newSource.getBatch(None, FileStreamSourceOffset(1)) } } + + test("SPARK-26629: multiple file sources work with restarts when a source does not have data") { + withTempDirs { case (dir, tmp) => + val sourceDir1 = new File(dir, "source1") + val sourceDir2 = new File(dir, "source2") + sourceDir1.mkdirs() + sourceDir2.mkdirs() + + val source1 = createFileStream("text", s"${sourceDir1.getCanonicalPath}") + val source2 = createFileStream("text", s"${sourceDir2.getCanonicalPath}") + val unioned = source1.union(source2) + + def addMultiTextFileData( + source1Content: String, + source2Content: String): StreamAction = { + val actions = Seq( + AddTextFileData(source1Content, sourceDir1, tmp), + AddTextFileData(source2Content, sourceDir2, tmp) + ).filter(_.content != null) // don't write to a source dir if no content specified + StreamProgressLockedActions(actions, desc = actions.mkString("[ ", " | ", " ]")) + } + + testStream(unioned)( + StartStream(), + addMultiTextFileData(source1Content = "source1_0", source2Content = "source2_0"), + CheckNewAnswer("source1_0", "source2_0"), + StopStream, + + StartStream(), + addMultiTextFileData(source1Content = "source1_1", source2Content = null), + CheckNewAnswer("source1_1"), + StopStream, + + // Restart after a batch with one file source having no new data. + // This restart is needed to hit the issue in SPARK-26629. + + StartStream(), + addMultiTextFileData(source1Content = null, source2Content = "source2_2"), + CheckNewAnswer("source2_2"), + StopStream, + + StartStream(), + addMultiTextFileData(source1Content = "source1_3", source2Content = "source2_3"), + CheckNewAnswer("source1_3", "source2_3"), + StopStream + ) + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 4d3a54a048e8..74ea0bfacba5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming.test import java.io.File +import java.util.ConcurrentModificationException import java.util.Locale import java.util.concurrent.TimeUnit @@ -651,4 +652,27 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { LastOptions.clear() } + + test("SPARK-26586: Streams should have isolated confs") { + import testImplicits._ + val input = MemoryStream[Int] + input.addData(1 to 10) + spark.conf.set("testKey1", 0) + val queries = (1 to 10).map { i => + spark.conf.set("testKey1", i) + input.toDF().writeStream + .foreachBatch { (df: Dataset[Row], id: Long) => + val v = df.sparkSession.conf.get("testKey1").toInt + if (i != v) { + throw new ConcurrentModificationException(s"Stream $i has the wrong conf value $v") + } + } + .start() + } + try { + queries.foreach(_.processAllAvailable()) + } finally { + queries.foreach(_.stop()) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala index e7e0ce64963a..8734639edaa6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -23,6 +23,7 @@ import org.scalatest.{BeforeAndAfterEach, Suite} import org.scalatest.concurrent.Eventually import org.apache.spark.{DebugFilesystem, SparkConf} +import org.apache.spark.internal.config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.internal.SQLConf @@ -38,7 +39,7 @@ trait SharedSparkSession protected def sparkConf = { new SparkConf() .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) - .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set(UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) .set(SQLConf.CODEGEN_FALLBACK.key, "false") // Disable ConvertToLocalRelation for better test coverage. Test cases built on // LocalRelation will exercise the optimization rules better by disabling it as diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 64d924fdb009..d1de9f037992 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -31,6 +31,7 @@ import org.apache.hive.service.server.HiveServer2 import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveUtils @@ -63,7 +64,7 @@ object HiveThriftServer2 extends Logging { server.start() listener = new HiveThriftServer2Listener(server, sqlContext.conf) sqlContext.sparkContext.addSparkListener(listener) - uiTab = if (sqlContext.sparkContext.getConf.getBoolean("spark.ui.enabled", true)) { + uiTab = if (sqlContext.sparkContext.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(sqlContext.sparkContext)) } else { None @@ -101,7 +102,7 @@ object HiveThriftServer2 extends Logging { logInfo("HiveThriftServer2 started") listener = new HiveThriftServer2Listener(server, SparkSQLEnv.sqlContext.conf) SparkSQLEnv.sparkContext.addSparkListener(listener) - uiTab = if (SparkSQLEnv.sparkContext.getConf.getBoolean("spark.ui.enabled", true)) { + uiTab = if (SparkSQLEnv.sparkContext.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(SparkSQLEnv.sparkContext)) } else { None diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 4f3914740ec2..132b0e4db0d7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -71,6 +71,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new ResolveHiveSerdeTable(session) +: new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: + new FallbackOrcDataSourceV2(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 4c2bc62b9faf..1db57b76ac24 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -33,6 +33,8 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, Dataset, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener @@ -59,8 +61,8 @@ object TestHive "org.apache.spark.sql.hive.execution.PairSerDe") .set("spark.sql.warehouse.dir", TestHiveContext.makeWarehouseDir().toURI.getPath) // SPARK-8910 - .set("spark.ui.enabled", "false") - .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set(UI_ENABLED, false) + .set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) // Disable ConvertToLocalRelation for better test coverage. Test cases built on // LocalRelation will exercise the optimization rules better by disabling it as // this rule may potentially block testing of other optimization rules such as diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index f1e842334416..dd0e1bd0fe30 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -206,7 +206,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { object PROCESS_TABLES extends QueryTest with SQLTestUtils { // Tests the latest version of every release line. - val testingVersions = Seq("2.2.2", "2.3.2", "2.4.0") + val testingVersions = Seq("2.3.2", "2.4.0") protected var spark: SparkSession = _ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index f839e8979d35..d3640086e74a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ @@ -335,7 +336,7 @@ object SetMetastoreURLTest extends Logging { val sparkConf = new SparkConf(loadDefaults = true) val builder = SparkSession.builder() .config(sparkConf) - .config("spark.ui.enabled", "false") + .config(UI_ENABLED.key, "false") .config("spark.sql.hive.metastore.version", "0.13.1") // The issue described in SPARK-16901 only appear when // spark.sql.hive.metastore.jars is not set to builtin. @@ -370,7 +371,7 @@ object SetWarehouseLocationTest extends Logging { def main(args: Array[String]): Unit = { TestUtils.configTestLog4j("INFO") - val sparkConf = new SparkConf(loadDefaults = true).set("spark.ui.enabled", "false") + val sparkConf = new SparkConf(loadDefaults = true).set(UI_ENABLED, false) val providedExpectedWarehouseLocation = sparkConf.getOption("spark.sql.test.expectedWarehouseDir") @@ -449,7 +450,7 @@ object TemporaryHiveUDFTest extends Logging { def main(args: Array[String]) { TestUtils.configTestLog4j("INFO") val conf = new SparkConf() - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) @@ -487,7 +488,7 @@ object PermanentHiveUDFTest1 extends Logging { def main(args: Array[String]) { TestUtils.configTestLog4j("INFO") val conf = new SparkConf() - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) @@ -525,7 +526,7 @@ object PermanentHiveUDFTest2 extends Logging { def main(args: Array[String]) { TestUtils.configTestLog4j("INFO") val conf = new SparkConf() - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) // Load a Hive UDF from the jar. @@ -561,7 +562,7 @@ object SparkSubmitClassLoaderTest extends Logging { TestUtils.configTestLog4j("INFO") val conf = new SparkConf() val hiveWarehouseLocation = Utils.createTempDir() - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) conf.set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) @@ -654,7 +655,7 @@ object SparkSQLConfTest extends Logging { // For this simple test, we do not really clone this object. override def clone: SparkConf = this } - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) // Run a simple command to make sure all lazy vals in hiveContext get instantiated. @@ -676,7 +677,7 @@ object SPARK_9757 extends QueryTest { new SparkConf() .set("spark.sql.hive.metastore.version", "0.13.1") .set("spark.sql.hive.metastore.jars", "maven") - .set("spark.ui.enabled", "false") + .set(UI_ENABLED, false) .set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString)) val hiveContext = new TestHiveContext(sparkContext) @@ -722,7 +723,7 @@ object SPARK_11009 extends QueryTest { val sparkContext = new SparkContext( new SparkConf() - .set("spark.ui.enabled", "false") + .set(UI_ENABLED, false) .set("spark.sql.shuffle.partitions", "100")) val hiveContext = new TestHiveContext(sparkContext) @@ -753,7 +754,7 @@ object SPARK_14244 extends QueryTest { val sparkContext = new SparkContext( new SparkConf() - .set("spark.ui.enabled", "false") + .set(UI_ENABLED, false) .set("spark.sql.shuffle.partitions", "100")) val hiveContext = new TestHiveContext(sparkContext) @@ -774,7 +775,7 @@ object SPARK_14244 extends QueryTest { object SPARK_18360 { def main(args: Array[String]): Unit = { val spark = SparkSession.builder() - .config("spark.ui.enabled", "false") + .config(UI_ENABLED.key, "false") .enableHiveSupport().getOrCreate() val defaultDbLocation = spark.catalog.getDatabase("default").locationUri diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index d93215fefb81..3402ed240f8b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -908,7 +908,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } assert(e.getMessage.contains( "The format of the existing table default.appendOrcToParquet is `ParquetFileFormat`. " + - "It doesn't match the specified format `OrcFileFormat`")) + "It doesn't match the specified format")) } withTable("appendParquetToJson") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index 07d8c5bacb1a..76134d23d18a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.execution import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.hive.test.TestHiveContext class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { @@ -27,7 +28,7 @@ class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { test("Multiple Hive Instances") { (1 to 10).map { i => val conf = new SparkConf() - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val ts = new TestHiveContext(new SparkContext("local", s"TestSQLContext$i", conf)) ts.sparkSession.sql("SHOW TABLES").collect() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index 94384185d190..6b2d0c656b37 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.Matchers._ + import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -25,7 +27,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.sql.functions.broadcast import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType @@ -91,4 +96,15 @@ class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with Te assert(size2 < tableStats.get.sizeInBytes) } } + + test("SPARK-26576 Broadcast hint not applied to partitioned table") { + withTable("tbl") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + spark.range(10).selectExpr("id", "id % 3 as p").write.partitionBy("p").saveAsTable("tbl") + val df = spark.table("tbl") + val qe = df.join(broadcast(df), "p").queryExecution + qe.sparkPlan.collect { case j: BroadcastHashJoinExec => j } should have size 1 + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 57b896612bfe..bf6d0ea5788d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.sources import java.io.File -import java.util.TimeZone import scala.util.Random @@ -126,61 +125,59 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } else { Seq(false) } - withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> "false") { - for (dataType <- supportedDataTypes) { - for (parquetDictionaryEncodingEnabled <- parquetDictionaryEncodingEnabledConfs) { - val extraMessage = if (isParquetDataSource) { - s" with parquet.enable.dictionary = $parquetDictionaryEncodingEnabled" - } else { - "" - } - logInfo(s"Testing $dataType data type$extraMessage") - - val extraOptions = Map[String, String]( - "parquet.enable.dictionary" -> parquetDictionaryEncodingEnabled.toString, - "timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SSSXXXXX" - ) - - withTempPath { file => - val path = file.getCanonicalPath - - val seed = System.nanoTime() - withClue(s"Random data generated with the seed: ${seed}") { - val dataGenerator = RandomDataGenerator.forType( - dataType = dataType, - nullable = true, - new Random(seed) - ).getOrElse { - fail(s"Failed to create data generator for schema $dataType") - } - - // Create a DF for the schema with random data. The index field is used to sort the - // DataFrame. This is a workaround for SPARK-10591. - val schema = new StructType() - .add("index", IntegerType, nullable = false) - .add("col", dataType, nullable = true) - val rdd = - spark.sparkContext.parallelize((1 to 10).map(i => Row(i, dataGenerator()))) - val df = spark.createDataFrame(rdd, schema).orderBy("index").coalesce(1) - - df.write - .mode("overwrite") - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .options(extraOptions) - .save(path) - - val loadedDF = spark - .read - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .schema(df.schema) - .options(extraOptions) - .load(path) - .orderBy("index") - - checkAnswer(loadedDF, df) + for (dataType <- supportedDataTypes) { + for (parquetDictionaryEncodingEnabled <- parquetDictionaryEncodingEnabledConfs) { + val extraMessage = if (isParquetDataSource) { + s" with parquet.enable.dictionary = $parquetDictionaryEncodingEnabled" + } else { + "" + } + logInfo(s"Testing $dataType data type$extraMessage") + + val extraOptions = Map[String, String]( + "parquet.enable.dictionary" -> parquetDictionaryEncodingEnabled.toString, + "timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SSSXXXXX" + ) + + withTempPath { file => + val path = file.getCanonicalPath + + val seed = System.nanoTime() + withClue(s"Random data generated with the seed: ${seed}") { + val dataGenerator = RandomDataGenerator.forType( + dataType = dataType, + nullable = true, + new Random(seed) + ).getOrElse { + fail(s"Failed to create data generator for schema $dataType") } + + // Create a DF for the schema with random data. The index field is used to sort the + // DataFrame. This is a workaround for SPARK-10591. + val schema = new StructType() + .add("index", IntegerType, nullable = false) + .add("col", dataType, nullable = true) + val rdd = + spark.sparkContext.parallelize((1 to 10).map(i => Row(i, dataGenerator()))) + val df = spark.createDataFrame(rdd, schema).orderBy("index").coalesce(1) + + df.write + .mode("overwrite") + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .options(extraOptions) + .save(path) + + val loadedDF = spark + .read + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .schema(df.schema) + .options(extraOptions) + .load(path) + .orderBy("index") + + checkAnswer(loadedDF, df) } } } diff --git a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js b/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js index d004f34ab186..5b75bc3011b6 100644 --- a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js +++ b/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js @@ -129,7 +129,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { svg.append("g") .attr("class", "x axis") .attr("transform", "translate(0," + height + ")") - .call(xAxis) + .call(xAxis); svg.append("g") .attr("class", "y axis") @@ -198,7 +198,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { lastClickedBatch = null; } lastClickedBatch = d.x; - highlightBatchRow(lastClickedBatch) + highlightBatchRow(lastClickedBatch); lastTimeout = window.setTimeout(function () { lastTimeout = null; if (lastClickedBatch != null) { @@ -261,9 +261,9 @@ function drawHistogram(id, values, minY, maxY, unitY, batchInterval) { svg.append("g") .attr("class", "y axis") - .call(yAxis) + .call(yAxis); - var bar = svg.selectAll(".bar") + svg.selectAll(".bar") .data(data) .enter() .append("g") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 135430f1ef62..e042adaa8611 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.streaming.scheduler.JobGenerator import org.apache.spark.util.Utils @@ -61,7 +62,7 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) "spark.yarn.principal", "spark.kerberos.keytab", "spark.kerberos.principal", - "spark.ui.filters", + UI_FILTERS.key, "spark.mesos.driver.frameworkId") val newSparkConf = new SparkConf(loadDefaults = false).setAll(sparkConfPairs) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 122f25b21a0d..c09cbb330844 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -38,6 +38,7 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.input.FixedLengthBinaryInputFormat import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.SerializationDebugger @@ -188,7 +189,7 @@ class StreamingContext private[streaming] ( private[streaming] val progressListener = new StreamingJobProgressListener(this) private[streaming] val uiTab: Option[StreamingTab] = - if (conf.getBoolean("spark.ui.enabled", true)) { + if (conf.get(UI_ENABLED)) { Some(new StreamingTab(this)) } else { None diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index fc6218a33f74..9d1203b7632c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkConf +import org.apache.spark.internal.config.UI._ import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.receiver._ @@ -200,7 +201,7 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { val sparkConf = new SparkConf() .setMaster("local[4]") // must be at least 3 as we are going to start 2 receivers .setAppName(framework) - .set("spark.ui.enabled", "true") + .set(UI_ENABLED, true) .set("spark.streaming.receiver.writeAheadLog.enable", "true") .set("spark.streaming.receiver.writeAheadLog.rollingIntervalSecs", "1") val batchDuration = Milliseconds(500) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 957feca2e552..29e451332088 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.ui.SparkUICssErrorHandler /** @@ -61,7 +62,7 @@ class UISeleniumSuite val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.ui.enabled", "true") + .set(UI_ENABLED, true) val ssc = new StreamingContext(conf, Seconds(1)) assert(ssc.sc.ui.isDefined, "Spark UI is not started!") ssc