diff --git a/.travis.yml b/.travis.yml deleted file mode 100644 index 05b94adeeb93..000000000000 --- a/.travis.yml +++ /dev/null @@ -1,50 +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. - -# Spark provides this Travis CI configuration file to help contributors -# check Scala/Java style conformance and JDK7/8 compilation easily -# during their preparing pull requests. -# - Scalastyle is executed during `maven install` implicitly. -# - Java Checkstyle is executed by `lint-java`. -# See the related discussion here. -# https://github.com/apache/spark/pull/12980 - -# 1. Choose OS (Ubuntu 14.04.3 LTS Server Edition 64bit, ~2 CORE, 7.5GB RAM) -sudo: required -dist: trusty - -# 2. Choose language and target JDKs for parallel builds. -language: java -jdk: - - oraclejdk8 - -# 3. Setup cache directory for SBT and Maven. -cache: - directories: - - $HOME/.sbt - - $HOME/.m2 - -# 4. Turn off notifications. -notifications: - email: false - -# 5. Run maven install before running lint-java. -install: - - export MAVEN_SKIP_RC=1 - - build/mvn -T 4 -q -DskipTests -Pkubernetes -Pmesos -Pyarn -Pkinesis-asl -Phive -Phive-thriftserver install - -# 6. Run lint-java. -script: - - dev/lint-java diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 2cb4cb8d531e..6a8fef5aa7b2 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -3473,13 +3473,21 @@ setMethod("collect_set", #' @details #' \code{split_string}: Splits string on regular expression. -#' Equivalent to \code{split} SQL function. +#' Equivalent to \code{split} SQL function. Optionally a +#' \code{limit} can be specified #' #' @rdname column_string_functions +#' @param limit determines the length of the returned array. +#' \itemize{ +#' \item \code{limit > 0}: length of the array will be at most \code{limit} +#' \item \code{limit <= 0}: the returned array can have any length +#' } +#' #' @aliases split_string split_string,Column-method #' @examples #' #' \dontrun{ +#' head(select(df, split_string(df$Class, "\\d", 2))) #' head(select(df, split_string(df$Sex, "a"))) #' head(select(df, split_string(df$Class, "\\d"))) #' # This is equivalent to the following SQL expression @@ -3487,8 +3495,9 @@ setMethod("collect_set", #' @note split_string 2.3.0 setMethod("split_string", signature(x = "Column", pattern = "character"), - function(x, pattern) { - jc <- callJStatic("org.apache.spark.sql.functions", "split", x@jc, pattern) + function(x, pattern, limit = -1) { + jc <- callJStatic("org.apache.spark.sql.functions", + "split", x@jc, pattern, as.integer(limit)) column(jc) }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 27c1b312d645..697d124095a7 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1258,7 +1258,7 @@ setGeneric("sort_array", function(x, asc = TRUE) { standardGeneric("sort_array") #' @rdname column_string_functions #' @name NULL -setGeneric("split_string", function(x, pattern) { standardGeneric("split_string") }) +setGeneric("split_string", function(x, pattern, ...) { standardGeneric("split_string") }) #' @rdname column_string_functions #' @name NULL diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 50eff3755edf..5cc75aa3f367 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1819,6 +1819,14 @@ test_that("string operators", { collect(select(df4, split_string(df4$a, "\\\\")))[1, 1], list(list("a.b@c.d 1", "b")) ) + expect_equal( + collect(select(df4, split_string(df4$a, "\\.", 2)))[1, 1], + list(list("a", "b@c.d 1\\b")) + ) + expect_equal( + collect(select(df4, split_string(df4$a, "b", 0)))[1, 1], + list(list("a.", "@c.d 1\\", "")) + ) l5 <- list(list(a = "abc")) df5 <- createDataFrame(l5) diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index d6371051ef7f..228494de6d5a 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -54,6 +54,8 @@ function build { img_path=$IMG_PATH --build-arg spark_jars=assembly/target/scala-$SPARK_SCALA_VERSION/jars + --build-arg + k8s_tests=resource-managers/kubernetes/integration-tests/tests ) else # Not passed as an argument to docker, but used to validate the Spark directory. diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java index bd8d9486acde..771a9541bb34 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java @@ -54,11 +54,8 @@ public final byte[] serialize(Object o) throws Exception { return ((String) o).getBytes(UTF_8); } else { ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - GZIPOutputStream out = new GZIPOutputStream(bytes); - try { + try (GZIPOutputStream out = new GZIPOutputStream(bytes)) { mapper.writeValue(out, o); - } finally { - out.close(); } return bytes.toByteArray(); } @@ -69,11 +66,8 @@ public final T deserialize(byte[] data, Class klass) throws Exception { if (klass.equals(String.class)) { return (T) new String(data, UTF_8); } else { - GZIPInputStream in = new GZIPInputStream(new ByteArrayInputStream(data)); - try { + try (GZIPInputStream in = new GZIPInputStream(new ByteArrayInputStream(data))) { return mapper.readValue(in, klass); - } finally { - in.close(); } } } diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java index 205f7df87c5b..39a952f2b0df 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java @@ -217,7 +217,7 @@ public void testSkip() throws Exception { public void testNegativeIndexValues() throws Exception { List expected = Arrays.asList(-100, -50, 0, 50, 100); - expected.stream().forEach(i -> { + expected.forEach(i -> { try { db.write(createCustomType1(i)); } catch (Exception e) { diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java index 056505ef5335..64fdb32a67ad 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java @@ -159,15 +159,21 @@ public void close() throws IOException { // accurately report the errors when they happen. RuntimeException error = null; byte[] dummy = new byte[8]; - try { - doCipherOp(encryptor, dummy, true); - } catch (Exception e) { - error = new RuntimeException(e); + if (encryptor != null) { + try { + doCipherOp(Cipher.ENCRYPT_MODE, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); + } + encryptor = null; } - try { - doCipherOp(decryptor, dummy, true); - } catch (Exception e) { - error = new RuntimeException(e); + if (decryptor != null) { + try { + doCipherOp(Cipher.DECRYPT_MODE, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); + } + decryptor = null; } random.close(); @@ -189,11 +195,11 @@ byte[] rawResponse(byte[] challenge) { } private byte[] decrypt(byte[] in) throws GeneralSecurityException { - return doCipherOp(decryptor, in, false); + return doCipherOp(Cipher.DECRYPT_MODE, in, false); } private byte[] encrypt(byte[] in) throws GeneralSecurityException { - return doCipherOp(encryptor, in, false); + return doCipherOp(Cipher.ENCRYPT_MODE, in, false); } private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key) @@ -205,11 +211,13 @@ private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key) byte[] iv = new byte[conf.ivLength()]; System.arraycopy(nonce, 0, iv, 0, Math.min(nonce.length, iv.length)); - encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); - encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv)); + CryptoCipher _encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + _encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv)); + this.encryptor = _encryptor; - decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); - decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv)); + CryptoCipher _decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + _decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv)); + this.decryptor = _decryptor; } /** @@ -241,29 +249,52 @@ private SecretKeySpec generateKey(String kdf, int iterations, byte[] salt, int k return new SecretKeySpec(key.getEncoded(), conf.keyAlgorithm()); } - private byte[] doCipherOp(CryptoCipher cipher, byte[] in, boolean isFinal) + private byte[] doCipherOp(int mode, byte[] in, boolean isFinal) throws GeneralSecurityException { - Preconditions.checkState(cipher != null); + CryptoCipher cipher; + switch (mode) { + case Cipher.ENCRYPT_MODE: + cipher = encryptor; + break; + case Cipher.DECRYPT_MODE: + cipher = decryptor; + break; + default: + throw new IllegalArgumentException(String.valueOf(mode)); + } - int scale = 1; - while (true) { - int size = in.length * scale; - byte[] buffer = new byte[size]; - try { - int outSize = isFinal ? cipher.doFinal(in, 0, in.length, buffer, 0) - : cipher.update(in, 0, in.length, buffer, 0); - if (outSize != buffer.length) { - byte[] output = new byte[outSize]; - System.arraycopy(buffer, 0, output, 0, output.length); - return output; - } else { - return buffer; + Preconditions.checkState(cipher != null, "Cipher is invalid because of previous error."); + + try { + int scale = 1; + while (true) { + int size = in.length * scale; + byte[] buffer = new byte[size]; + try { + int outSize = isFinal ? cipher.doFinal(in, 0, in.length, buffer, 0) + : cipher.update(in, 0, in.length, buffer, 0); + if (outSize != buffer.length) { + byte[] output = new byte[outSize]; + System.arraycopy(buffer, 0, output, 0, output.length); + return output; + } else { + return buffer; + } + } catch (ShortBufferException e) { + // Try again with a bigger buffer. + scale *= 2; } - } catch (ShortBufferException e) { - // Try again with a bigger buffer. - scale *= 2; } + } catch (InternalError ie) { + // SPARK-25535. The commons-cryto library will throw InternalError if something goes wrong, + // and leave bad state behind in the Java wrappers, so it's not safe to use them afterwards. + if (mode == Cipher.ENCRYPT_MODE) { + this.encryptor = null; + } else { + this.decryptor = null; + } + throw ie; } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java index b64e4b7a970b..2745052265f7 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java @@ -107,45 +107,72 @@ public void addToChannel(Channel ch) throws IOException { private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { private final ByteArrayWritableChannel byteChannel; private final CryptoOutputStream cos; + private boolean isCipherValid; EncryptionHandler(TransportCipher cipher) throws IOException { byteChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); cos = cipher.createOutputStream(byteChannel); + isCipherValid = true; } @Override public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { - ctx.write(new EncryptedMessage(cos, msg, byteChannel), promise); + ctx.write(new EncryptedMessage(this, cos, msg, byteChannel), promise); } @Override public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { try { - cos.close(); + if (isCipherValid) { + cos.close(); + } } finally { super.close(ctx, promise); } } + + /** + * SPARK-25535. Workaround for CRYPTO-141. Avoid further interaction with the underlying cipher + * after an error occurs. + */ + void reportError() { + this.isCipherValid = false; + } + + boolean isCipherValid() { + return isCipherValid; + } } private static class DecryptionHandler extends ChannelInboundHandlerAdapter { private final CryptoInputStream cis; private final ByteArrayReadableChannel byteChannel; + private boolean isCipherValid; DecryptionHandler(TransportCipher cipher) throws IOException { byteChannel = new ByteArrayReadableChannel(); cis = cipher.createInputStream(byteChannel); + isCipherValid = true; } @Override public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { + if (!isCipherValid) { + throw new IOException("Cipher is in invalid state."); + } byteChannel.feedData((ByteBuf) data); byte[] decryptedData = new byte[byteChannel.readableBytes()]; int offset = 0; while (offset < decryptedData.length) { - offset += cis.read(decryptedData, offset, decryptedData.length - offset); + // SPARK-25535: workaround for CRYPTO-141. + try { + offset += cis.read(decryptedData, offset, decryptedData.length - offset); + } catch (InternalError ie) { + isCipherValid = false; + throw ie; + } } ctx.fireChannelRead(Unpooled.wrappedBuffer(decryptedData, 0, decryptedData.length)); @@ -154,7 +181,9 @@ public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { try { - cis.close(); + if (isCipherValid) { + cis.close(); + } } finally { super.channelInactive(ctx); } @@ -165,8 +194,9 @@ private static class EncryptedMessage extends AbstractFileRegion { private final boolean isByteBuf; private final ByteBuf buf; private final FileRegion region; + private final CryptoOutputStream cos; + private final EncryptionHandler handler; private long transferred; - private CryptoOutputStream cos; // Due to streaming issue CRYPTO-125: https://issues.apache.org/jira/browse/CRYPTO-125, it has // to utilize two helper ByteArrayWritableChannel for streaming. One is used to receive raw data @@ -176,9 +206,14 @@ private static class EncryptedMessage extends AbstractFileRegion { private ByteBuffer currentEncrypted; - EncryptedMessage(CryptoOutputStream cos, Object msg, ByteArrayWritableChannel ch) { + EncryptedMessage( + EncryptionHandler handler, + CryptoOutputStream cos, + Object msg, + ByteArrayWritableChannel ch) { Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion, "Unrecognized message type: %s", msg.getClass().getName()); + this.handler = handler; this.isByteBuf = msg instanceof ByteBuf; this.buf = isByteBuf ? (ByteBuf) msg : null; this.region = isByteBuf ? null : (FileRegion) msg; @@ -261,6 +296,9 @@ public long transferTo(WritableByteChannel target, long position) throws IOExcep } private void encryptMore() throws IOException { + if (!handler.isCipherValid()) { + throw new IOException("Cipher is in invalid state."); + } byteRawChannel.reset(); if (isByteBuf) { @@ -269,8 +307,14 @@ private void encryptMore() throws IOException { } else { region.transferTo(byteRawChannel, region.transferred()); } - cos.write(byteRawChannel.getData(), 0, byteRawChannel.length()); - cos.flush(); + + try { + cos.write(byteRawChannel.getData(), 0, byteRawChannel.length()); + cos.flush(); + } catch (InternalError ie) { + handler.reportError(); + throw ie; + } currentEncrypted = ByteBuffer.wrap(byteEncChannel.getData(), 0, byteEncChannel.length()); 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 6d5cccd20b33..43a6bc7dc3d0 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 @@ -296,17 +296,21 @@ public long maxChunksBeingTransferred() { * and could take long time to process due to disk contentions. By configuring a slightly * higher number of shuffler server threads, we are able to reserve some threads for * handling other RPC messages, thus making the Client less likely to experience timeout - * when sending RPC messages to the shuffle server. Default to 0, which is 2*#cores - * or io.serverThreads. 90 would mean 90% of 2*#cores or 90% of io.serverThreads - * which equals 0.9 * 2*#cores or 0.9 * io.serverThreads. + * when sending RPC messages to the shuffle server. The number of threads used for handling + * chunked fetch requests are percentage of io.serverThreads (if defined) else it is a percentage + * of 2 * #cores. However, a percentage of 0 means netty default number of threads which + * is 2 * #cores ignoring io.serverThreads. The percentage here is configured via + * spark.shuffle.server.chunkFetchHandlerThreadsPercent. The returned value is rounded off to + * ceiling of the nearest integer. */ public int chunkFetchHandlerThreads() { if (!this.getModuleName().equalsIgnoreCase("shuffle")) { return 0; } int chunkFetchHandlerThreadsPercent = - conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 0); - return this.serverThreads() > 0 ? (this.serverThreads() * chunkFetchHandlerThreadsPercent)/100: - (2 * NettyRuntime.availableProcessors() * chunkFetchHandlerThreadsPercent)/100; + conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 100); + return (int)Math.ceil( + (this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors()) * + chunkFetchHandlerThreadsPercent/(double)100); } } 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 824482af08dd..37a8664a5266 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 @@ -143,37 +143,39 @@ public void releaseBuffers() { } private FetchResult fetchChunks(List chunkIndices) throws Exception { - TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); - final Semaphore sem = new Semaphore(0); - final FetchResult res = new FetchResult(); - res.successChunks = Collections.synchronizedSet(new HashSet()); - res.failedChunks = Collections.synchronizedSet(new HashSet()); - res.buffers = Collections.synchronizedList(new LinkedList()); - ChunkReceivedCallback callback = new ChunkReceivedCallback() { - @Override - public void onSuccess(int chunkIndex, ManagedBuffer buffer) { - buffer.retain(); - res.successChunks.add(chunkIndex); - res.buffers.add(buffer); - sem.release(); - } + try (TransportClient client = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort())) { + final Semaphore sem = new Semaphore(0); + + res.successChunks = Collections.synchronizedSet(new HashSet()); + res.failedChunks = Collections.synchronizedSet(new HashSet()); + res.buffers = Collections.synchronizedList(new LinkedList()); + + ChunkReceivedCallback callback = new ChunkReceivedCallback() { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + buffer.retain(); + res.successChunks.add(chunkIndex); + res.buffers.add(buffer); + sem.release(); + } - @Override - public void onFailure(int chunkIndex, Throwable e) { - res.failedChunks.add(chunkIndex); - sem.release(); - } - }; + @Override + public void onFailure(int chunkIndex, Throwable e) { + res.failedChunks.add(chunkIndex); + sem.release(); + } + }; - for (int chunkIndex : chunkIndices) { - client.fetchChunk(STREAM_ID, chunkIndex, callback); - } - if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { - fail("Timeout getting response from the server"); + for (int chunkIndex : chunkIndices) { + client.fetchChunk(STREAM_ID, chunkIndex, callback); + } + if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } } - client.close(); return res; } diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java index a3519fe4a423..c0aa298a4017 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java @@ -18,8 +18,11 @@ package org.apache.spark.network.crypto; import java.util.Arrays; +import java.util.Map; +import java.security.InvalidKeyException; import static java.nio.charset.StandardCharsets.UTF_8; +import com.google.common.collect.ImmutableMap; import org.junit.BeforeClass; import org.junit.Test; import static org.junit.Assert.*; @@ -104,4 +107,18 @@ public void testBadChallenge() throws Exception { challenge.cipher, challenge.keyLength, challenge.nonce, badChallenge)); } + @Test(expected = InvalidKeyException.class) + public void testBadKeySize() throws Exception { + Map mconf = ImmutableMap.of("spark.network.crypto.keyLength", "42"); + TransportConf conf = new TransportConf("rpc", new MapConfigProvider(mconf)); + + try (AuthEngine engine = new AuthEngine("appId", "secret", conf)) { + engine.challenge(); + fail("Should have failed to create challenge message."); + + // Call close explicitly to make sure it's idempotent. + engine.close(); + } + } + } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java index 386738ece51a..371149bef397 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java @@ -37,14 +37,8 @@ public ShuffleIndexInformation(File indexFile) throws IOException { size = (int)indexFile.length(); ByteBuffer buffer = ByteBuffer.allocate(size); offsets = buffer.asLongBuffer(); - DataInputStream dis = null; - try { - dis = new DataInputStream(Files.newInputStream(indexFile.toPath())); + try (DataInputStream dis = new DataInputStream(Files.newInputStream(indexFile.toPath()))) { dis.readFully(buffer.array()); - } finally { - if (dis != null) { - dis.close(); - } } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index d2072a54fa41..459629c5f05f 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -98,19 +98,19 @@ public void testSortShuffleBlocks() throws IOException { resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); - InputStream block0Stream = - resolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream(); - String block0 = CharStreams.toString( - new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); - block0Stream.close(); - assertEquals(sortBlock0, block0); - - InputStream block1Stream = - resolver.getBlockData("app0", "exec0", 0, 0, 1).createInputStream(); - String block1 = CharStreams.toString( - new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); - block1Stream.close(); - assertEquals(sortBlock1, block1); + try (InputStream block0Stream = resolver.getBlockData( + "app0", "exec0", 0, 0, 0).createInputStream()) { + String block0 = + CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); + assertEquals(sortBlock0, block0); + } + + try (InputStream block1Stream = resolver.getBlockData( + "app0", "exec0", 0, 0, 1).createInputStream()) { + String block1 = + CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); + assertEquals(sortBlock1, block1); + } } @Test @@ -149,7 +149,7 @@ public void testNormalizeAndInternPathname() { private void assertPathsMatch(String p1, String p2, String p3, String expectedPathname) { String normPathname = - ExternalShuffleBlockResolver.createNormalizedInternedPathname(p1, p2, p3); + ExternalShuffleBlockResolver.createNormalizedInternedPathname(p1, p2, p3); assertEquals(expectedPathname, normPathname); File file = new File(normPathname); String returnedPath = file.getPath(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index a6a1b8d0ac3f..526b96b36447 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -133,37 +133,37 @@ private FetchResult fetchBlocks( final Semaphore requestsRemaining = new Semaphore(0); - ExternalShuffleClient client = new ExternalShuffleClient(clientConf, null, false, 5000); - client.init(APP_ID); - client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, - new BlockFetchingListener() { - @Override - public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { - synchronized (this) { - if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { - data.retain(); - res.successBlocks.add(blockId); - res.buffers.add(data); - requestsRemaining.release(); + try (ExternalShuffleClient client = new ExternalShuffleClient(clientConf, null, false, 5000)) { + client.init(APP_ID); + client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, + new BlockFetchingListener() { + @Override + public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { + synchronized (this) { + if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { + data.retain(); + res.successBlocks.add(blockId); + res.buffers.add(data); + requestsRemaining.release(); + } } } - } - - @Override - public void onBlockFetchFailure(String blockId, Throwable exception) { - synchronized (this) { - if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { - res.failedBlocks.add(blockId); - requestsRemaining.release(); + + @Override + public void onBlockFetchFailure(String blockId, Throwable exception) { + synchronized (this) { + if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { + res.failedBlocks.add(blockId); + requestsRemaining.release(); + } } } - } - }, null); + }, null); - if (!requestsRemaining.tryAcquire(blockIds.length, 5, TimeUnit.SECONDS)) { - fail("Timeout getting response from the server"); + if (!requestsRemaining.tryAcquire(blockIds.length, 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } } - client.close(); return res; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index 16bad9f1b319..82caf392b821 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -96,14 +96,16 @@ private void validate(String appId, String secretKey, boolean encrypt) ImmutableMap.of("spark.authenticate.enableSaslEncryption", "true"))); } - ExternalShuffleClient client = - new ExternalShuffleClient(testConf, new TestSecretKeyHolder(appId, secretKey), true, 5000); - client.init(appId); - // Registration either succeeds or throws an exception. - client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), "exec0", - new ExecutorShuffleInfo(new String[0], 0, - "org.apache.spark.shuffle.sort.SortShuffleManager")); - client.close(); + try (ExternalShuffleClient client = + new ExternalShuffleClient( + testConf, new TestSecretKeyHolder(appId, secretKey), true, 5000)) { + client.init(appId); + // Registration either succeeds or throws an exception. + client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), "exec0", + new ExecutorShuffleInfo( + new String[0], 0, "org.apache.spark.shuffle.sort.SortShuffleManager") + ); + } } /** Provides a secret key holder which always returns the given secret key, for a single appId. */ diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java index f7c22dddb8cc..06a248c9a27c 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java @@ -191,10 +191,9 @@ public static CountMinSketch readFrom(InputStream in) throws IOException { * Reads in a {@link CountMinSketch} from a byte array. */ public static CountMinSketch readFrom(byte[] bytes) throws IOException { - InputStream in = new ByteArrayInputStream(bytes); - CountMinSketch cms = readFrom(in); - in.close(); - return cms; + try (InputStream in = new ByteArrayInputStream(bytes)) { + return readFrom(in); + } } /** diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java index fd1906d2e5ae..b78c1677a121 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java @@ -322,10 +322,10 @@ public void writeTo(OutputStream out) throws IOException { @Override public byte[] toByteArray() throws IOException { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - writeTo(out); - out.close(); - return out.toByteArray(); + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + writeTo(out); + return out.toByteArray(); + } } public static CountMinSketchImpl readFrom(InputStream in) throws IOException { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index dff4a73f3e9d..3a3bfc4a94bb 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -958,6 +958,12 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { } public UTF8String[] split(UTF8String pattern, int limit) { + // Java String's split method supports "ignore empty string" behavior when the limit is 0 + // whereas other languages do not. To avoid this java specific behavior, we fall back to + // -1 when the limit is 0. + if (limit == 0) { + limit = -1; + } String[] splits = toString().split(pattern.toString(), limit); UTF8String[] res = new UTF8String[splits.length]; for (int i = 0; i < res.length; i++) { diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index dae13f03b02f..cf9cc6b1800a 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -393,12 +393,14 @@ public void substringSQL() { @Test public void split() { - assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), -1), - new UTF8String[]{fromString("ab"), fromString("def"), fromString("ghi")})); - assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), 2), - new UTF8String[]{fromString("ab"), fromString("def,ghi")})); - assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), 2), - new UTF8String[]{fromString("ab"), fromString("def,ghi")})); + UTF8String[] negativeAndZeroLimitCase = + new UTF8String[]{fromString("ab"), fromString("def"), fromString("ghi"), fromString("")}; + assertTrue(Arrays.equals(fromString("ab,def,ghi,").split(fromString(","), 0), + negativeAndZeroLimitCase)); + assertTrue(Arrays.equals(fromString("ab,def,ghi,").split(fromString(","), -1), + negativeAndZeroLimitCase)); + assertTrue(Arrays.equals(fromString("ab,def,ghi,").split(fromString(","), 2), + new UTF8String[]{fromString("ab"), fromString("def,ghi,")})); } @Test diff --git a/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java b/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java index 0cced9e22295..2e18715b600e 100644 --- a/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java +++ b/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java @@ -135,62 +135,58 @@ private void readAsync() throws IOException { } finally { stateChangeLock.unlock(); } - executorService.execute(new Runnable() { - - @Override - public void run() { - stateChangeLock.lock(); - try { - if (isClosed) { - readInProgress = false; - return; - } - // Flip this so that the close method will not close the underlying input stream when we - // are reading. - isReading = true; - } finally { - stateChangeLock.unlock(); + executorService.execute(() -> { + stateChangeLock.lock(); + try { + if (isClosed) { + readInProgress = false; + return; } + // Flip this so that the close method will not close the underlying input stream when we + // are reading. + isReading = true; + } finally { + stateChangeLock.unlock(); + } - // Please note that it is safe to release the lock and read into the read ahead buffer - // because either of following two conditions will hold - 1. The active buffer has - // data available to read so the reader will not read from the read ahead buffer. - // 2. This is the first time read is called or the active buffer is exhausted, - // in that case the reader waits for this async read to complete. - // So there is no race condition in both the situations. - int read = 0; - int off = 0, len = arr.length; - Throwable exception = null; - try { - // try to fill the read ahead buffer. - // if a reader is waiting, possibly return early. - do { - read = underlyingInputStream.read(arr, off, len); - if (read <= 0) break; - off += read; - len -= read; - } while (len > 0 && !isWaiting.get()); - } catch (Throwable ex) { - exception = ex; - if (ex instanceof Error) { - // `readException` may not be reported to the user. Rethrow Error to make sure at least - // The user can see Error in UncaughtExceptionHandler. - throw (Error) ex; - } - } finally { - stateChangeLock.lock(); - readAheadBuffer.limit(off); - if (read < 0 || (exception instanceof EOFException)) { - endOfStream = true; - } else if (exception != null) { - readAborted = true; - readException = exception; - } - readInProgress = false; - signalAsyncReadComplete(); - stateChangeLock.unlock(); - closeUnderlyingInputStreamIfNecessary(); + // Please note that it is safe to release the lock and read into the read ahead buffer + // because either of following two conditions will hold - 1. The active buffer has + // data available to read so the reader will not read from the read ahead buffer. + // 2. This is the first time read is called or the active buffer is exhausted, + // in that case the reader waits for this async read to complete. + // So there is no race condition in both the situations. + int read = 0; + int off = 0, len = arr.length; + Throwable exception = null; + try { + // try to fill the read ahead buffer. + // if a reader is waiting, possibly return early. + do { + read = underlyingInputStream.read(arr, off, len); + if (read <= 0) break; + off += read; + len -= read; + } while (len > 0 && !isWaiting.get()); + } catch (Throwable ex) { + exception = ex; + if (ex instanceof Error) { + // `readException` may not be reported to the user. Rethrow Error to make sure at least + // The user can see Error in UncaughtExceptionHandler. + throw (Error) ex; } + } finally { + stateChangeLock.lock(); + readAheadBuffer.limit(off); + if (read < 0 || (exception instanceof EOFException)) { + endOfStream = true; + } else if (exception != null) { + readAborted = true; + readException = exception; + } + readInProgress = false; + signalAsyncReadComplete(); + stateChangeLock.unlock(); + closeUnderlyingInputStreamIfNecessary(); } }); } 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 323a5d3c5283..b020a6d99247 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 @@ -152,9 +152,9 @@ public void write(Iterator> records) throws IOException { } for (int i = 0; i < numPartitions; i++) { - final DiskBlockObjectWriter writer = partitionWriters[i]; - partitionWriterSegments[i] = writer.commitAndGet(); - writer.close(); + try (DiskBlockObjectWriter writer = partitionWriters[i]) { + partitionWriterSegments[i] = writer.commitAndGet(); + } } File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); 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 c7d2db4217d9..1c0d664afb13 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 @@ -181,42 +181,43 @@ private void writeSortedFile(boolean isLastFile) { // around this, we pass a dummy no-op serializer. final SerializerInstance ser = DummySerializerInstance.INSTANCE; - final DiskBlockObjectWriter writer = - blockManager.getDiskWriter(blockId, file, ser, fileBufferSizeBytes, writeMetricsToUse); - int currentPartition = -1; - final int uaoSize = UnsafeAlignedOffset.getUaoSize(); - while (sortedRecords.hasNext()) { - sortedRecords.loadNext(); - final int partition = sortedRecords.packedRecordPointer.getPartitionId(); - assert (partition >= currentPartition); - if (partition != currentPartition) { - // Switch to the new partition - if (currentPartition != -1) { - final FileSegment fileSegment = writer.commitAndGet(); - spillInfo.partitionLengths[currentPartition] = fileSegment.length(); + final FileSegment committedSegment; + try (DiskBlockObjectWriter writer = + blockManager.getDiskWriter(blockId, file, ser, fileBufferSizeBytes, writeMetricsToUse)) { + + final int uaoSize = UnsafeAlignedOffset.getUaoSize(); + while (sortedRecords.hasNext()) { + sortedRecords.loadNext(); + final int partition = sortedRecords.packedRecordPointer.getPartitionId(); + assert (partition >= currentPartition); + if (partition != currentPartition) { + // Switch to the new partition + if (currentPartition != -1) { + final FileSegment fileSegment = writer.commitAndGet(); + spillInfo.partitionLengths[currentPartition] = fileSegment.length(); + } + currentPartition = partition; } - currentPartition = partition; - } - final long recordPointer = sortedRecords.packedRecordPointer.getRecordPointer(); - final Object recordPage = taskMemoryManager.getPage(recordPointer); - final long recordOffsetInPage = taskMemoryManager.getOffsetInPage(recordPointer); - int dataRemaining = UnsafeAlignedOffset.getSize(recordPage, recordOffsetInPage); - long recordReadPosition = recordOffsetInPage + uaoSize; // skip over record length - while (dataRemaining > 0) { - final int toTransfer = Math.min(diskWriteBufferSize, dataRemaining); - Platform.copyMemory( - recordPage, recordReadPosition, writeBuffer, Platform.BYTE_ARRAY_OFFSET, toTransfer); - writer.write(writeBuffer, 0, toTransfer); - recordReadPosition += toTransfer; - dataRemaining -= toTransfer; + final long recordPointer = sortedRecords.packedRecordPointer.getRecordPointer(); + final Object recordPage = taskMemoryManager.getPage(recordPointer); + final long recordOffsetInPage = taskMemoryManager.getOffsetInPage(recordPointer); + int dataRemaining = UnsafeAlignedOffset.getSize(recordPage, recordOffsetInPage); + long recordReadPosition = recordOffsetInPage + uaoSize; // skip over record length + while (dataRemaining > 0) { + final int toTransfer = Math.min(diskWriteBufferSize, dataRemaining); + Platform.copyMemory( + recordPage, recordReadPosition, writeBuffer, Platform.BYTE_ARRAY_OFFSET, toTransfer); + writer.write(writeBuffer, 0, toTransfer); + recordReadPosition += toTransfer; + dataRemaining -= toTransfer; + } + writer.recordWritten(); } - writer.recordWritten(); - } - final FileSegment committedSegment = writer.commitAndGet(); - writer.close(); + committedSegment = writer.commitAndGet(); + } // If `writeSortedFile()` was called from `closeAndGetSpills()` and no records were inserted, // then the file might be empty. Note that it might be better to avoid calling // writeSortedFile() in that case. 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 12c056af9a51..b1254e08fa50 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 @@ -83,7 +83,7 @@ $(function() { collapseTablePageLoad('collapse-aggregated-rdds','aggregated-rdds'); collapseTablePageLoad('collapse-aggregated-activeBatches','aggregated-activeBatches'); collapseTablePageLoad('collapse-aggregated-completedBatches','aggregated-completedBatches'); - collapseTablePageLoad('collapse-aggregated-runningExecutions','runningExecutions'); - collapseTablePageLoad('collapse-aggregated-completedExecutions','completedExecutions'); - collapseTablePageLoad('collapse-aggregated-failedExecutions','failedExecutions'); + collapseTablePageLoad('collapse-aggregated-runningExecutions','aggregated-runningExecutions'); + collapseTablePageLoad('collapse-aggregated-completedExecutions','aggregated-completedExecutions'); + collapseTablePageLoad('collapse-aggregated-failedExecutions','aggregated-failedExecutions'); }); \ No newline at end of file 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 d5f2865f8728..61b379f28680 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -927,8 +927,6 @@ object SparkSubmit extends CommandLineUtils with Logging { } catch { case e: SparkUserAppException => exitFn(e.exitCode) - case e: SparkException => - printErrorAndExit(e.getMessage()) } } 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 7722db56ee29..0664c5ac752c 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -154,72 +154,19 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { */ @DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { - val version = SnappyCompressionCodec.version - override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt - new SnappyOutputStreamWrapper(new SnappyOutputStream(s, blockSize)) - } - - override def compressedInputStream(s: InputStream): InputStream = new SnappyInputStream(s) -} - -/** - * Object guards against memory leak bug in snappy-java library: - * (https://github.com/xerial/snappy-java/issues/131). - * Before a new version of the library, we only call the method once and cache the result. - */ -private final object SnappyCompressionCodec { - private lazy val version: String = try { + try { Snappy.getNativeLibraryVersion } catch { case e: Error => throw new IllegalArgumentException(e) } -} -/** - * Wrapper over `SnappyOutputStream` which guards against write-after-close and double-close - * issues. See SPARK-7660 for more details. This wrapping can be removed if we upgrade to a version - * of snappy-java that contains the fix for https://github.com/xerial/snappy-java/issues/107. - */ -private final class SnappyOutputStreamWrapper(os: SnappyOutputStream) extends OutputStream { - - private[this] var closed: Boolean = false - - override def write(b: Int): Unit = { - if (closed) { - throw new IOException("Stream is closed") - } - os.write(b) - } - - override def write(b: Array[Byte]): Unit = { - if (closed) { - throw new IOException("Stream is closed") - } - os.write(b) - } - - override def write(b: Array[Byte], off: Int, len: Int): Unit = { - if (closed) { - throw new IOException("Stream is closed") - } - os.write(b, off, len) - } - - override def flush(): Unit = { - if (closed) { - throw new IOException("Stream is closed") - } - os.flush() + override def compressedOutputStream(s: OutputStream): OutputStream = { + val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt + new SnappyOutputStream(s, blockSize) } - override def close(): Unit = { - if (!closed) { - closed = true - os.close() - } - } + override def compressedInputStream(s: InputStream): InputStream = new SnappyInputStream(s) } /** diff --git a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala index 00621976b77f..18b735b8035a 100644 --- a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala +++ b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.security -import java.io.{InputStream, OutputStream} +import java.io.{Closeable, InputStream, IOException, OutputStream} import java.nio.ByteBuffer import java.nio.channels.{ReadableByteChannel, WritableByteChannel} import java.util.Properties @@ -54,8 +54,10 @@ private[spark] object CryptoStreamUtils extends Logging { val params = new CryptoParams(key, sparkConf) val iv = createInitializationVector(params.conf) os.write(iv) - new CryptoOutputStream(params.transformation, params.conf, os, params.keySpec, - new IvParameterSpec(iv)) + new ErrorHandlingOutputStream( + new CryptoOutputStream(params.transformation, params.conf, os, params.keySpec, + new IvParameterSpec(iv)), + os) } /** @@ -70,8 +72,10 @@ private[spark] object CryptoStreamUtils extends Logging { val helper = new CryptoHelperChannel(channel) helper.write(ByteBuffer.wrap(iv)) - new CryptoOutputStream(params.transformation, params.conf, helper, params.keySpec, - new IvParameterSpec(iv)) + new ErrorHandlingWritableChannel( + new CryptoOutputStream(params.transformation, params.conf, helper, params.keySpec, + new IvParameterSpec(iv)), + helper) } /** @@ -84,8 +88,10 @@ private[spark] object CryptoStreamUtils extends Logging { val iv = new Array[Byte](IV_LENGTH_IN_BYTES) ByteStreams.readFully(is, iv) val params = new CryptoParams(key, sparkConf) - new CryptoInputStream(params.transformation, params.conf, is, params.keySpec, - new IvParameterSpec(iv)) + new ErrorHandlingInputStream( + new CryptoInputStream(params.transformation, params.conf, is, params.keySpec, + new IvParameterSpec(iv)), + is) } /** @@ -100,8 +106,10 @@ private[spark] object CryptoStreamUtils extends Logging { JavaUtils.readFully(channel, buf) val params = new CryptoParams(key, sparkConf) - new CryptoInputStream(params.transformation, params.conf, channel, params.keySpec, - new IvParameterSpec(iv)) + new ErrorHandlingReadableChannel( + new CryptoInputStream(params.transformation, params.conf, channel, params.keySpec, + new IvParameterSpec(iv)), + channel) } def toCryptoConf(conf: SparkConf): Properties = { @@ -157,6 +165,117 @@ private[spark] object CryptoStreamUtils extends Logging { } + /** + * SPARK-25535. The commons-cryto library will throw InternalError if something goes + * wrong, and leave bad state behind in the Java wrappers, so it's not safe to use them + * afterwards. This wrapper detects that situation and avoids further calls into the + * commons-crypto code, while still allowing the underlying streams to be closed. + * + * This should be removed once CRYPTO-141 is fixed (and Spark upgrades its commons-crypto + * dependency). + */ + trait BaseErrorHandler extends Closeable { + + private var closed = false + + /** The encrypted stream that may get into an unhealthy state. */ + protected def cipherStream: Closeable + + /** + * The underlying stream that is being wrapped by the encrypted stream, so that it can be + * closed even if there's an error in the crypto layer. + */ + protected def original: Closeable + + protected def safeCall[T](fn: => T): T = { + if (closed) { + throw new IOException("Cipher stream is closed.") + } + try { + fn + } catch { + case ie: InternalError => + closed = true + original.close() + throw ie + } + } + + override def close(): Unit = { + if (!closed) { + cipherStream.close() + } + } + + } + + // Visible for testing. + class ErrorHandlingReadableChannel( + protected val cipherStream: ReadableByteChannel, + protected val original: ReadableByteChannel) + extends ReadableByteChannel with BaseErrorHandler { + + override def read(src: ByteBuffer): Int = safeCall { + cipherStream.read(src) + } + + override def isOpen(): Boolean = cipherStream.isOpen() + + } + + private class ErrorHandlingInputStream( + protected val cipherStream: InputStream, + protected val original: InputStream) + extends InputStream with BaseErrorHandler { + + override def read(b: Array[Byte]): Int = safeCall { + cipherStream.read(b) + } + + override def read(b: Array[Byte], off: Int, len: Int): Int = safeCall { + cipherStream.read(b, off, len) + } + + override def read(): Int = safeCall { + cipherStream.read() + } + } + + private class ErrorHandlingWritableChannel( + protected val cipherStream: WritableByteChannel, + protected val original: WritableByteChannel) + extends WritableByteChannel with BaseErrorHandler { + + override def write(src: ByteBuffer): Int = safeCall { + cipherStream.write(src) + } + + override def isOpen(): Boolean = cipherStream.isOpen() + + } + + private class ErrorHandlingOutputStream( + protected val cipherStream: OutputStream, + protected val original: OutputStream) + extends OutputStream with BaseErrorHandler { + + override def flush(): Unit = safeCall { + cipherStream.flush() + } + + override def write(b: Array[Byte]): Unit = safeCall { + cipherStream.write(b) + } + + override def write(b: Array[Byte], off: Int, len: Int): Unit = safeCall { + cipherStream.write(b, off, len) + } + + override def write(b: Int): Unit = safeCall { + cipherStream.write(b) + } + } + private class CryptoParams(key: Array[Byte], sparkConf: SparkConf) { val keySpec = new SecretKeySpec(key, "AES") diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index e237281c552b..9839cbb99f86 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -112,10 +112,12 @@ private[spark] class AppStatusStore( } } - def stageAttempt(stageId: Int, stageAttemptId: Int, details: Boolean = false): v1.StageData = { + def stageAttempt(stageId: Int, stageAttemptId: Int, + details: Boolean = false): (v1.StageData, Seq[Int]) = { val stageKey = Array(stageId, stageAttemptId) - val stage = store.read(classOf[StageDataWrapper], stageKey).info - if (details) stageWithDetails(stage) else stage + val stageDataWrapper = store.read(classOf[StageDataWrapper], stageKey) + val stage = if (details) stageWithDetails(stageDataWrapper.info) else stageDataWrapper.info + (stage, stageDataWrapper.jobIds.toSeq) } def taskCount(stageId: Int, stageAttemptId: Int): Long = { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 96249e4bfd5f..30d52b97833e 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -56,7 +56,7 @@ private[v1] class StagesResource extends BaseAppResource { @PathParam("stageAttemptId") stageAttemptId: Int, @QueryParam("details") @DefaultValue("true") details: Boolean): StageData = withUI { ui => try { - ui.store.stageAttempt(stageId, stageAttemptId, details = details) + ui.store.stageAttempt(stageId, stageAttemptId, details = details)._1 } catch { case _: NoSuchElementException => // Change the message depending on whether there are any attempts for the requested stage. 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 7428bbe6c559..0f74b07a6265 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 @@ -105,7 +105,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We val stageAttemptId = parameterAttempt.toInt val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" - val stageData = parent.store + val (stageData, stageJobIds) = parent.store .asOption(parent.store.stageAttempt(stageId, stageAttemptId, details = false)) .getOrElse { val content = @@ -183,6 +183,15 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We {Utils.bytesToString(stageData.diskBytesSpilled)} }} + {if (!stageJobIds.isEmpty) { +
  • + Associated Job Ids: + {stageJobIds.map(jobId => {val detailUrl = "%s/jobs/job/?id=%s".format( + UIUtils.prependBaseUri(request, parent.basePath), jobId) + {s"${jobId}"}    + })} +
  • + }} @@ -1048,7 +1057,7 @@ private[ui] object ApiHelper { } def lastStageNameAndDescription(store: AppStatusStore, job: JobData): (String, String) = { - val stage = store.asOption(store.stageAttempt(job.stageIds.max, 0)) + val stage = store.asOption(store.stageAttempt(job.stageIds.max, 0)._1) (stage.map(_.name).getOrElse(""), stage.flatMap(_.description).getOrElse(job.name)) } diff --git a/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java index a6589d289814..40a7c9486ae5 100644 --- a/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java +++ b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java @@ -39,30 +39,28 @@ public void setUp() throws ClassNotFoundException, SQLException { sc = new JavaSparkContext("local", "JavaAPISuite"); Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); - Connection connection = - DriverManager.getConnection("jdbc:derby:target/JavaJdbcRDDSuiteDb;create=true"); - try { - Statement create = connection.createStatement(); - create.execute( - "CREATE TABLE FOO(" + - "ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1)," + - "DATA INTEGER)"); - create.close(); + try (Connection connection = DriverManager.getConnection( + "jdbc:derby:target/JavaJdbcRDDSuiteDb;create=true")) { + + try (Statement create = connection.createStatement()) { + create.execute( + "CREATE TABLE FOO(ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY" + + " (START WITH 1, INCREMENT BY 1), DATA INTEGER)"); + } - PreparedStatement insert = connection.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)"); - for (int i = 1; i <= 100; i++) { - insert.setInt(1, i * 2); - insert.executeUpdate(); + try (PreparedStatement insert = connection.prepareStatement( + "INSERT INTO FOO(DATA) VALUES(?)")) { + for (int i = 1; i <= 100; i++) { + insert.setInt(1, i * 2); + insert.executeUpdate(); + } } - insert.close(); } catch (SQLException e) { // If table doesn't exist... if (e.getSQLState().compareTo("X0Y32") != 0) { throw e; } - } finally { - connection.close(); } } 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 0d5c5ea7903e..a07d0e84ea85 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 @@ -186,14 +186,14 @@ private List> readRecordsFromFile() throws IOException { if (conf.getBoolean("spark.shuffle.compress", true)) { in = CompressionCodec$.MODULE$.createCodec(conf).compressedInputStream(in); } - DeserializationStream recordsStream = serializer.newInstance().deserializeStream(in); - Iterator> records = recordsStream.asKeyValueIterator(); - while (records.hasNext()) { - Tuple2 record = records.next(); - assertEquals(i, hashPartitioner.getPartition(record._1())); - recordsList.add(record); + try (DeserializationStream recordsStream = serializer.newInstance().deserializeStream(in)) { + Iterator> records = recordsStream.asKeyValueIterator(); + while (records.hasNext()) { + Tuple2 record = records.next(); + assertEquals(i, hashPartitioner.getPartition(record._1())); + recordsList.add(record); + } } - recordsStream.close(); startOffset += partitionSize; } } diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index 01b5fb7b4668..3992ab7049bd 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -997,10 +997,10 @@ public void binaryFiles() throws Exception { FileOutputStream fos1 = new FileOutputStream(file1); - FileChannel channel1 = fos1.getChannel(); - ByteBuffer bbuf = ByteBuffer.wrap(content1); - channel1.write(bbuf); - channel1.close(); + try (FileChannel channel1 = fos1.getChannel()) { + ByteBuffer bbuf = ByteBuffer.wrap(content1); + channel1.write(bbuf); + } JavaPairRDD readRDD = sc.binaryFiles(tempDirName, 3); List> result = readRDD.collect(); for (Tuple2 res : result) { @@ -1018,10 +1018,10 @@ public void binaryFilesCaching() throws Exception { FileOutputStream fos1 = new FileOutputStream(file1); - FileChannel channel1 = fos1.getChannel(); - ByteBuffer bbuf = ByteBuffer.wrap(content1); - channel1.write(bbuf); - channel1.close(); + try (FileChannel channel1 = fos1.getChannel()) { + ByteBuffer bbuf = ByteBuffer.wrap(content1); + channel1.write(bbuf); + } JavaPairRDD readRDD = sc.binaryFiles(tempDirName).cache(); readRDD.foreach(pair -> pair._2().toArray()); // force the file to read @@ -1042,13 +1042,12 @@ public void binaryRecords() throws Exception { FileOutputStream fos1 = new FileOutputStream(file1); - FileChannel channel1 = fos1.getChannel(); - - for (int i = 0; i < numOfCopies; i++) { - ByteBuffer bbuf = ByteBuffer.wrap(content1); - channel1.write(bbuf); + try (FileChannel channel1 = fos1.getChannel()) { + for (int i = 0; i < numOfCopies; i++) { + ByteBuffer bbuf = ByteBuffer.wrap(content1); + channel1.write(bbuf); + } } - channel1.close(); JavaRDD readRDD = sc.binaryRecords(tempDirName, content1.length); assertEquals(numOfCopies,readRDD.count()); 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 9eae3605d073..652c36ffa6e7 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -74,20 +74,25 @@ trait TestPrematureExit { @volatile var exitedCleanly = false mainObject.exitFn = (_) => exitedCleanly = true + @volatile var exception: Exception = null val thread = new Thread { override def run() = try { mainObject.main(input) } catch { - // If exceptions occur after the "exit" has happened, fine to ignore them. - // These represent code paths not reachable during normal execution. - case e: Exception => if (!exitedCleanly) throw e + // Capture the exception to check whether the exception contains searchString or not + case e: Exception => exception = e } } thread.start() thread.join() - val joined = printStream.lineBuffer.mkString("\n") - if (!joined.contains(searchString)) { - fail(s"Search string '$searchString' not found in $joined") + if (exitedCleanly) { + val joined = printStream.lineBuffer.mkString("\n") + assert(joined.contains(searchString)) + } else { + assert(exception != null) + if (!exception.getMessage.contains(searchString)) { + throw exception + } } } } 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 78f618f8a216..0d3611c80b8d 100644 --- a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala @@ -16,13 +16,16 @@ */ package org.apache.spark.security -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream} -import java.nio.channels.Channels +import java.io._ +import java.nio.ByteBuffer +import java.nio.channels.{Channels, ReadableByteChannel} import java.nio.charset.StandardCharsets.UTF_8 import java.nio.file.Files import java.util.{Arrays, Random, UUID} import com.google.common.io.ByteStreams +import org.mockito.Matchers.any +import org.mockito.Mockito._ import org.apache.spark._ import org.apache.spark.internal.config._ @@ -164,6 +167,36 @@ class CryptoStreamUtilsSuite extends SparkFunSuite { } } + test("error handling wrapper") { + val wrapped = mock(classOf[ReadableByteChannel]) + val decrypted = mock(classOf[ReadableByteChannel]) + val errorHandler = new CryptoStreamUtils.ErrorHandlingReadableChannel(decrypted, wrapped) + + when(decrypted.read(any(classOf[ByteBuffer]))) + .thenThrow(new IOException()) + .thenThrow(new InternalError()) + .thenReturn(1) + + val out = ByteBuffer.allocate(1) + intercept[IOException] { + errorHandler.read(out) + } + intercept[InternalError] { + errorHandler.read(out) + } + + val e = intercept[IOException] { + errorHandler.read(out) + } + assert(e.getMessage().contains("is closed")) + errorHandler.close() + + verify(decrypted, times(2)).read(any(classOf[ByteBuffer])) + verify(wrapped, never()).read(any(classOf[ByteBuffer])) + verify(decrypted, never()).close() + verify(wrapped, times(1)).close() + } + private def createConf(extra: (String, String)*): SparkConf = { val conf = new SparkConf() extra.foreach { case (k, v) => conf.set(k, v) } diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index 8a04b621f8ce..c91882851847 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -95,7 +95,8 @@ $env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=512m" Pop-Location # ========================== Hadoop bin package -$hadoopVer = "2.6.4" +# This must match the version at https://github.com/steveloughran/winutils/tree/master/hadoop-2.7.1 +$hadoopVer = "2.7.1" $hadoopPath = "$tools\hadoop" if (!(Test-Path $hadoopPath)) { New-Item -ItemType Directory -Force -Path $hadoopPath | Out-Null diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index cce5f8b6975c..b80f55de98e1 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -114,7 +114,7 @@ PUBLISH_SCALA_2_10=0 SCALA_2_10_PROFILES="-Pscala-2.10" SCALA_2_11_PROFILES= if [[ $SPARK_VERSION > "2.3" ]]; then - BASE_PROFILES="$BASE_PROFILES -Pkubernetes -Pflume" + BASE_PROFILES="$BASE_PROFILES -Pkubernetes" SCALA_2_11_PROFILES="-Pkafka-0-8" else PUBLISH_SCALA_2_10=1 @@ -191,9 +191,19 @@ if [[ "$1" == "package" ]]; then make_binary_release() { NAME=$1 FLAGS="$MVN_EXTRA_OPTS -B $BASE_RELEASE_PROFILES $2" + # BUILD_PACKAGE can be "withpip", "withr", or both as "withpip,withr" BUILD_PACKAGE=$3 SCALA_VERSION=$4 + PIP_FLAG="" + if [[ $BUILD_PACKAGE == *"withpip"* ]]; then + PIP_FLAG="--pip" + fi + R_FLAG="" + if [[ $BUILD_PACKAGE == *"withr"* ]]; then + R_FLAG="--r" + fi + # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds # share the same Zinc server. ZINC_PORT=$((ZINC_PORT + 1)) @@ -217,18 +227,13 @@ if [[ "$1" == "package" ]]; then # Get maven home set by MVN MVN_HOME=`$MVN -version 2>&1 | grep 'Maven home' | awk '{print $NF}'` + echo "Creating distribution" + ./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz \ + $PIP_FLAG $R_FLAG $FLAGS \ + -DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log + cd .. - if [ -z "$BUILD_PACKAGE" ]; then - echo "Creating distribution without PIP/R package" - ./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz $FLAGS \ - -DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log - cd .. - elif [[ "$BUILD_PACKAGE" == "withr" ]]; then - echo "Creating distribution with R package" - ./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz --r $FLAGS \ - -DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log - cd .. - + if [[ -n $R_FLAG ]]; then echo "Copying and signing R source package" R_DIST_NAME=SparkR_$SPARK_VERSION.tar.gz cp spark-$SPARK_VERSION-bin-$NAME/R/$R_DIST_NAME . @@ -239,12 +244,9 @@ if [[ "$1" == "package" ]]; then echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \ SHA512 $R_DIST_NAME > \ $R_DIST_NAME.sha512 - else - echo "Creating distribution with PIP package" - ./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz --pip $FLAGS \ - -DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log - cd .. + fi + if [[ -n $PIP_FLAG ]]; then echo "Copying and signing python distribution" PYTHON_DIST_NAME=pyspark-$PYSPARK_VERSION.tar.gz cp spark-$SPARK_VERSION-bin-$NAME/python/dist/$PYTHON_DIST_NAME . @@ -277,8 +279,10 @@ if [[ "$1" == "package" ]]; then declare -A BINARY_PKGS_ARGS BINARY_PKGS_ARGS["hadoop2.7"]="-Phadoop-2.7 $HIVE_PROFILES" if ! is_dry_run; then - BINARY_PKGS_ARGS["hadoop2.6"]="-Phadoop-2.6 $HIVE_PROFILES" BINARY_PKGS_ARGS["without-hadoop"]="-Phadoop-provided" + if [[ $SPARK_VERSION < "3.0." ]]; then + BINARY_PKGS_ARGS["hadoop2.6"]="-Phadoop-2.6 $HIVE_PROFILES" + fi if [[ $SPARK_VERSION < "2.2." ]]; then BINARY_PKGS_ARGS["hadoop2.4"]="-Phadoop-2.4 $HIVE_PROFILES" BINARY_PKGS_ARGS["hadoop2.3"]="-Phadoop-2.3 $HIVE_PROFILES" @@ -286,10 +290,7 @@ if [[ "$1" == "package" ]]; then fi declare -A BINARY_PKGS_EXTRA - BINARY_PKGS_EXTRA["hadoop2.7"]="withpip" - if ! is_dry_run; then - BINARY_PKGS_EXTRA["hadoop2.6"]="withr" - fi + BINARY_PKGS_EXTRA["hadoop2.7"]="withpip,withr" echo "Packages to build: ${!BINARY_PKGS_ARGS[@]}" for key in ${!BINARY_PKGS_ARGS[@]}; do diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 deleted file mode 100644 index 2dcab8533b01..000000000000 --- a/dev/deps/spark-deps-hadoop-2.6 +++ /dev/null @@ -1,198 +0,0 @@ -JavaEWAH-0.3.2.jar -RoaringBitmap-0.5.11.jar -ST4-4.0.4.jar -activation-1.1.1.jar -aircompressor-0.10.jar -antlr-2.7.7.jar -antlr-runtime-3.4.jar -antlr4-runtime-4.7.jar -aopalliance-1.0.jar -aopalliance-repackaged-2.4.0-b34.jar -apache-log4j-extras-1.2.17.jar -apacheds-i18n-2.0.0-M15.jar -apacheds-kerberos-codec-2.0.0-M15.jar -api-asn1-api-1.0.0-M20.jar -api-util-1.0.0-M20.jar -arpack_combined_all-0.1.jar -arrow-format-0.10.0.jar -arrow-memory-0.10.0.jar -arrow-vector-0.10.0.jar -automaton-1.11-8.jar -avro-1.8.2.jar -avro-ipc-1.8.2.jar -avro-mapred-1.8.2-hadoop2.jar -bonecp-0.8.0.RELEASE.jar -breeze-macros_2.11-0.13.2.jar -breeze_2.11-0.13.2.jar -calcite-avatica-1.2.0-incubating.jar -calcite-core-1.2.0-incubating.jar -calcite-linq4j-1.2.0-incubating.jar -chill-java-0.9.3.jar -chill_2.11-0.9.3.jar -commons-beanutils-1.7.0.jar -commons-beanutils-core-1.8.0.jar -commons-cli-1.2.jar -commons-codec-1.10.jar -commons-collections-3.2.2.jar -commons-compiler-3.0.10.jar -commons-compress-1.8.1.jar -commons-configuration-1.6.jar -commons-crypto-1.0.0.jar -commons-dbcp-1.4.jar -commons-digester-1.8.jar -commons-httpclient-3.1.jar -commons-io-2.4.jar -commons-lang-2.6.jar -commons-lang3-3.5.jar -commons-logging-1.1.3.jar -commons-math3-3.4.1.jar -commons-net-3.1.jar -commons-pool-1.5.4.jar -compress-lzf-1.0.3.jar -core-1.1.2.jar -curator-client-2.6.0.jar -curator-framework-2.6.0.jar -curator-recipes-2.6.0.jar -datanucleus-api-jdo-3.2.6.jar -datanucleus-core-3.2.10.jar -datanucleus-rdbms-3.2.9.jar -derby-10.12.1.1.jar -eigenbase-properties-1.1.5.jar -flatbuffers-1.2.0-3f79e055.jar -generex-1.0.1.jar -gson-2.2.4.jar -guava-14.0.1.jar -guice-3.0.jar -guice-servlet-3.0.jar -hadoop-annotations-2.6.5.jar -hadoop-auth-2.6.5.jar -hadoop-client-2.6.5.jar -hadoop-common-2.6.5.jar -hadoop-hdfs-2.6.5.jar -hadoop-mapreduce-client-app-2.6.5.jar -hadoop-mapreduce-client-common-2.6.5.jar -hadoop-mapreduce-client-core-2.6.5.jar -hadoop-mapreduce-client-jobclient-2.6.5.jar -hadoop-mapreduce-client-shuffle-2.6.5.jar -hadoop-yarn-api-2.6.5.jar -hadoop-yarn-client-2.6.5.jar -hadoop-yarn-common-2.6.5.jar -hadoop-yarn-server-common-2.6.5.jar -hadoop-yarn-server-web-proxy-2.6.5.jar -hk2-api-2.4.0-b34.jar -hk2-locator-2.4.0-b34.jar -hk2-utils-2.4.0-b34.jar -hppc-0.7.2.jar -htrace-core-3.0.4.jar -httpclient-4.5.6.jar -httpcore-4.4.10.jar -ivy-2.4.0.jar -jackson-annotations-2.9.6.jar -jackson-core-2.9.6.jar -jackson-core-asl-1.9.13.jar -jackson-databind-2.9.6.jar -jackson-dataformat-yaml-2.9.6.jar -jackson-jaxrs-1.9.13.jar -jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations-2.9.6.jar -jackson-module-paranamer-2.9.6.jar -jackson-module-scala_2.11-2.9.6.jar -jackson-xc-1.9.13.jar -janino-3.0.10.jar -javassist-3.18.1-GA.jar -javax.annotation-api-1.2.jar -javax.inject-1.jar -javax.inject-2.4.0-b34.jar -javax.servlet-api-3.1.0.jar -javax.ws.rs-api-2.0.1.jar -javolution-5.5.1.jar -jaxb-api-2.2.2.jar -jcl-over-slf4j-1.7.16.jar -jdo-api-3.0.1.jar -jersey-client-2.22.2.jar -jersey-common-2.22.2.jar -jersey-container-servlet-2.22.2.jar -jersey-container-servlet-core-2.22.2.jar -jersey-guava-2.22.2.jar -jersey-media-jaxb-2.22.2.jar -jersey-server-2.22.2.jar -jetty-6.1.26.jar -jetty-util-6.1.26.jar -jline-2.14.6.jar -joda-time-2.9.3.jar -jodd-core-3.5.2.jar -jpam-1.1.jar -json4s-ast_2.11-3.5.3.jar -json4s-core_2.11-3.5.3.jar -json4s-jackson_2.11-3.5.3.jar -json4s-scalap_2.11-3.5.3.jar -jsr305-1.3.9.jar -jta-1.1.jar -jtransforms-2.4.0.jar -jul-to-slf4j-1.7.16.jar -kryo-shaded-4.0.2.jar -kubernetes-client-3.0.0.jar -kubernetes-model-2.0.0.jar -leveldbjni-all-1.8.jar -libfb303-0.9.3.jar -libthrift-0.9.3.jar -log4j-1.2.17.jar -logging-interceptor-3.8.1.jar -lz4-java-1.4.0.jar -machinist_2.11-0.6.1.jar -macro-compat_2.11-1.1.1.jar -mesos-1.4.0-shaded-protobuf.jar -metrics-core-3.1.5.jar -metrics-graphite-3.1.5.jar -metrics-json-3.1.5.jar -metrics-jvm-3.1.5.jar -minlog-1.3.0.jar -netty-3.9.9.Final.jar -netty-all-4.1.17.Final.jar -objenesis-2.5.1.jar -okhttp-3.8.1.jar -okio-1.13.0.jar -opencsv-2.3.jar -orc-core-1.5.2-nohive.jar -orc-mapreduce-1.5.2-nohive.jar -orc-shims-1.5.2.jar -oro-2.0.8.jar -osgi-resource-locator-1.0.1.jar -paranamer-2.8.jar -parquet-column-1.10.0.jar -parquet-common-1.10.0.jar -parquet-encoding-1.10.0.jar -parquet-format-2.4.0.jar -parquet-hadoop-1.10.0.jar -parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.10.0.jar -protobuf-java-2.5.0.jar -py4j-0.10.7.jar -pyrolite-4.13.jar -scala-compiler-2.11.12.jar -scala-library-2.11.12.jar -scala-parser-combinators_2.11-1.1.0.jar -scala-reflect-2.11.12.jar -scala-xml_2.11-1.0.5.jar -shapeless_2.11-2.3.2.jar -slf4j-api-1.7.16.jar -slf4j-log4j12-1.7.16.jar -snakeyaml-1.18.jar -snappy-0.2.jar -snappy-java-1.1.7.1.jar -spire-macros_2.11-0.13.0.jar -spire_2.11-0.13.0.jar -stax-api-1.0-2.jar -stax-api-1.0.1.jar -stream-2.7.0.jar -stringtemplate-3.2.1.jar -super-csv-2.2.0.jar -univocity-parsers-2.7.3.jar -validation-api-1.1.0.Final.jar -xbean-asm6-shaded-4.8.jar -xercesImpl-2.9.1.jar -xmlenc-0.52.jar -xz-1.5.jar -zjsonpatch-0.3.0.jar -zookeeper-3.4.6.jar -zstd-jni-1.3.2-2.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index d1d695c47c0b..3b17f88a82c1 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -139,7 +139,7 @@ libfb303-0.9.3.jar libthrift-0.9.3.jar log4j-1.2.17.jar logging-interceptor-3.8.1.jar -lz4-java-1.4.0.jar +lz4-java-1.5.0.jar machinist_2.11-0.6.1.jar macro-compat_2.11-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar @@ -154,9 +154,9 @@ objenesis-2.5.1.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.2-nohive.jar -orc-mapreduce-1.5.2-nohive.jar -orc-shims-1.5.2.jar +orc-core-1.5.3-nohive.jar +orc-mapreduce-1.5.3-nohive.jar +orc-shims-1.5.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index e9691eb02aba..c818b2c39f74 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -154,7 +154,7 @@ libfb303-0.9.3.jar libthrift-0.9.3.jar log4j-1.2.17.jar logging-interceptor-3.8.1.jar -lz4-java-1.4.0.jar +lz4-java-1.5.0.jar machinist_2.11-0.6.1.jar macro-compat_2.11-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar @@ -172,9 +172,9 @@ okhttp-2.7.5.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.2-nohive.jar -orc-mapreduce-1.5.2-nohive.jar -orc-shims-1.5.2.jar +orc-core-1.5.3-nohive.jar +orc-mapreduce-1.5.3-nohive.jar +orc-shims-1.5.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar diff --git a/dev/mima b/dev/mima index cd2694ff4d3d..a9ac8aff11eb 100755 --- a/dev/mima +++ b/dev/mima @@ -24,7 +24,7 @@ set -e FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" -SPARK_PROFILES="-Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Pflume -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive" +SPARK_PROFILES="-Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive" TOOLS_CLASSPATH="$(build/sbt -DcopyDependencies=false "export tools/fullClasspath" | tail -n1)" OLD_DEPS_CLASSPATH="$(build/sbt -DcopyDependencies=false $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)" diff --git a/dev/run-tests.py b/dev/run-tests.py index f534637b80d6..a125f5b07b9c 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -305,7 +305,6 @@ def get_hadoop_profiles(hadoop_version): """ sbt_maven_hadoop_profiles = { - "hadoop2.6": ["-Phadoop-2.6"], "hadoop2.7": ["-Phadoop-2.7"], } @@ -334,7 +333,6 @@ def build_spark_sbt(hadoop_version): build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags sbt_goals = ["test:package", # Build test jars as some tests depend on them "streaming-kafka-0-8-assembly/assembly", - "streaming-flume-assembly/assembly", "streaming-kinesis-asl-assembly/assembly"] profiles_and_goals = build_profiles + sbt_goals @@ -369,15 +367,7 @@ def build_spark_assembly_sbt(hadoop_version, checkstyle=False): if checkstyle: run_java_style_checks() - # Note that we skip Unidoc build only if Hadoop 2.6 is explicitly set in this SBT build. - # Due to a different dependency resolution in SBT & Unidoc by an unknown reason, the - # documentation build fails on a specific machine & environment in Jenkins but it was unable - # to reproduce. Please see SPARK-20343. This is a band-aid fix that should be removed in - # the future. - is_hadoop_version_2_6 = os.environ.get("AMPLAB_JENKINS_BUILD_PROFILE") == "hadoop2.6" - if not is_hadoop_version_2_6: - # Make sure that Java and Scala API documentation can be generated - build_spark_unidoc_sbt(hadoop_version) + build_spark_unidoc_sbt(hadoop_version) def build_apache_spark(build_tool, hadoop_version): @@ -528,14 +518,14 @@ def main(): # if we're on the Amplab Jenkins build servers setup variables # to reflect the environment settings build_tool = os.environ.get("AMPLAB_JENKINS_BUILD_TOOL", "sbt") - hadoop_version = os.environ.get("AMPLAB_JENKINS_BUILD_PROFILE", "hadoop2.6") + hadoop_version = os.environ.get("AMPLAB_JENKINS_BUILD_PROFILE", "hadoop2.7") test_env = "amplab_jenkins" # add path for Python3 in Jenkins if we're calling from a Jenkins machine os.environ["PATH"] = "/home/anaconda/envs/py3k/bin:" + os.environ.get("PATH") else: # else we're running locally and can use local settings build_tool = "sbt" - hadoop_version = os.environ.get("HADOOP_PROFILE", "hadoop2.6") + hadoop_version = os.environ.get("HADOOP_PROFILE", "hadoop2.7") test_env = "local" print("[info] Using build tool", build_tool, "with Hadoop profile", hadoop_version, diff --git a/dev/sbt-checkstyle b/dev/sbt-checkstyle index 8821a7c0e4cc..1e825dbf78a1 100755 --- a/dev/sbt-checkstyle +++ b/dev/sbt-checkstyle @@ -26,7 +26,6 @@ ERRORS=$(echo -e "q\n" \ -Pkafka-0-8 \ -Pkubernetes \ -Pyarn \ - -Pflume \ -Phive \ -Phive-thriftserver \ checkstyle test:checkstyle \ diff --git a/dev/scalastyle b/dev/scalastyle index b8053df05fa2..0448e1dd74d1 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -26,9 +26,9 @@ ERRORS=$(echo -e "q\n" \ -Pkafka-0-8 \ -Pkubernetes \ -Pyarn \ - -Pflume \ -Phive \ -Phive-thriftserver \ + -Pspark-ganglia-lgpl \ scalastyle test:scalastyle \ | awk '{if($1~/error/)print}' \ ) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index e267fbfa623b..bd5f00916668 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -283,56 +283,6 @@ def __hash__(self): ] ) -streaming_flume_sink = Module( - name="streaming-flume-sink", - dependencies=[streaming], - source_file_regexes=[ - "external/flume-sink", - ], - build_profile_flags=[ - "-Pflume", - ], - environ={ - "ENABLE_FLUME_TESTS": "1" - }, - sbt_test_goals=[ - "streaming-flume-sink/test", - ] -) - - -streaming_flume = Module( - name="streaming-flume", - dependencies=[streaming], - source_file_regexes=[ - "external/flume", - ], - build_profile_flags=[ - "-Pflume", - ], - environ={ - "ENABLE_FLUME_TESTS": "1" - }, - sbt_test_goals=[ - "streaming-flume/test", - ] -) - - -streaming_flume_assembly = Module( - name="streaming-flume-assembly", - dependencies=[streaming_flume, streaming_flume_sink], - source_file_regexes=[ - "external/flume-assembly", - ], - build_profile_flags=[ - "-Pflume", - ], - environ={ - "ENABLE_FLUME_TESTS": "1" - } -) - mllib_local = Module( name="mllib-local", @@ -425,14 +375,12 @@ def __hash__(self): pyspark_core, streaming, streaming_kafka, - streaming_flume_assembly, streaming_kinesis_asl ], source_file_regexes=[ "python/pyspark/streaming" ], environ={ - "ENABLE_FLUME_TESTS": "1", "ENABLE_KAFKA_0_8_TESTS": "1" }, python_test_goals=[ @@ -557,6 +505,16 @@ def __hash__(self): sbt_test_goals=["kubernetes/test"] ) + +spark_ganglia_lgpl = Module( + name="spark-ganglia-lgpl", + dependencies=[], + build_profile_flags=["-Pspark-ganglia-lgpl"], + source_file_regexes=[ + "external/spark-ganglia-lgpl", + ] +) + # The root module is a dummy module which is used to run all of the tests. # No other modules should directly depend on this module. root = Module( diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 2fbd6b5e98f7..cc8f5d3a8e3a 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -29,10 +29,9 @@ export LC_ALL=C # TODO: This would be much nicer to do in SBT, once SBT supports Maven-style resolution. # NOTE: These should match those in the release publishing script -HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Pflume -Phive" +HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Phive" MVN="build/mvn" HADOOP_PROFILES=( - hadoop-2.6 hadoop-2.7 hadoop-3.1 ) diff --git a/docs/building-spark.md b/docs/building-spark.md index 1501f0bb8454..55830d38a9e2 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -49,25 +49,20 @@ To create a Spark distribution like those distributed by the to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured with Maven profile settings and so on like the direct Maven build. Example: - ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes + ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes This will build Spark distribution along with Python pip and R packages. For more information on usage, run `./dev/make-distribution.sh --help` ## Specifying the Hadoop Version and Enabling YARN You can specify the exact version of Hadoop to compile against through the `hadoop.version` property. -If unset, Spark will build against Hadoop 2.6.X by default. You can enable the `yarn` profile and optionally set the `yarn.version` property if it is different from `hadoop.version`. -Examples: +Example: - # Apache Hadoop 2.6.X - ./build/mvn -Pyarn -DskipTests clean package - - # Apache Hadoop 2.7.X and later - ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.3 -DskipTests clean package + ./build/mvn -Pyarn -Dhadoop.version=2.8.5 -DskipTests clean package ## Building With Hive and JDBC Support @@ -104,13 +99,6 @@ Note: Kafka 0.8 support is deprecated as of Spark 2.3.0. Kafka 0.10 support is still automatically built. -## Building with Flume support - -Apache Flume support must be explicitly enabled with the `flume` profile. -Note: Flume support is deprecated as of Spark 2.3.0. - - ./build/mvn -Pflume -DskipTests clean package - ## Building submodules individually It's possible to build Spark submodules using the `mvn -pl` option. diff --git a/docs/configuration.md b/docs/configuration.md index 55773937d4d7..613e214783d5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -954,7 +954,7 @@ Apart from these, the following properties are also available, and may be useful org.apache.spark.io.LZ4CompressionCodec, org.apache.spark.io.LZFCompressionCodec, org.apache.spark.io.SnappyCompressionCodec, - and org.apache.spark.io.ZstdCompressionCodec. + and org.apache.spark.io.ZStdCompressionCodec. diff --git a/docs/index.md b/docs/index.md index 40f628b794c0..d269f54c7343 100644 --- a/docs/index.md +++ b/docs/index.md @@ -30,9 +30,6 @@ Spark runs on Java 8+, Python 2.7+/3.4+ and R 3.1+. For the Scala API, Spark {{s uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version ({{site.SCALA_BINARY_VERSION}}.x). -Note that support for Java 7, Python 2.6 and old Hadoop versions before 2.6.5 were removed as of Spark 2.2.0. -Support for Scala 2.10 was removed as of 2.3.0. - # Running the Examples and Shell Spark comes with several sample programs. Scala, Java, Python and R examples are in the diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 687f9e46c328..bdf7b99966e4 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -396,8 +396,7 @@ To use a custom metrics.properties for the application master and executors, upd and those log files will be aggregated in a rolling fashion. This will be used with YARN's rolling log aggregation, to enable this feature in YARN side yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds should be - configured in yarn-site.xml. - This feature can only be used with Hadoop 2.6.4+. The Spark log4j appender needs be changed to use + configured in yarn-site.xml. The Spark log4j appender needs be changed to use FileAppender or another appender that can handle the files being removed while it is running. Based on the file name configured in the log4j configuration (like spark.log), the user should set the regex (spark*) to include all the log files that need to be aggregated. diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a1d7b1108bf7..fb03ed2e292b 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1890,6 +1890,10 @@ working with timestamps in `pandas_udf`s to get the best performance, see # Migration Guide +## Upgrading From Spark SQL 2.4 to 3.0 + + - In PySpark, when creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 3.0, the builder come to not update the configurations. This is the same behavior as Java/Scala API in 2.3 and above. If you want to update them, you need to update them prior to creating a `SparkSession`. + ## Upgrading From Spark SQL 2.3 to 2.4 - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. @@ -1973,6 +1977,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. + - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. ## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above @@ -2135,6 +2140,8 @@ working with timestamps in `pandas_udf`s to get the best performance, see - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. + - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 2.3, the builder come to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. + ## Upgrading From Spark SQL 2.1 to 2.2 - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 44ae52e81cd6..a83ebd9449fa 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -4,7 +4,7 @@ title: Spark Streaming Custom Receivers --- Spark Streaming can receive streaming data from any arbitrary data source beyond -the ones for which it has built-in support (that is, beyond Flume, Kafka, Kinesis, files, sockets, etc.). +the ones for which it has built-in support (that is, beyond Kafka, Kinesis, files, sockets, etc.). This requires the developer to implement a *receiver* that is customized for receiving data from the concerned data source. This guide walks through the process of implementing a custom receiver and using it in a Spark Streaming application. Note that custom receivers can be implemented diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md deleted file mode 100644 index a1b6942ffe0a..000000000000 --- a/docs/streaming-flume-integration.md +++ /dev/null @@ -1,169 +0,0 @@ ---- -layout: global -title: Spark Streaming + Flume Integration Guide ---- - -[Apache Flume](https://flume.apache.org/) is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. Here we explain how to configure Flume and Spark Streaming to receive data from Flume. There are two approaches to this. - -**Note: Flume support is deprecated as of Spark 2.3.0.** - -## Approach 1: Flume-style Push-based Approach -Flume is designed to push data between Flume agents. In this approach, Spark Streaming essentially sets up a receiver that acts an Avro agent for Flume, to which Flume can push the data. Here are the configuration steps. - -#### General Requirements -Choose a machine in your cluster such that - -- When your Flume + Spark Streaming application is launched, one of the Spark workers must run on that machine. - -- Flume can be configured to push data to a port on that machine. - -Due to the push model, the streaming application needs to be up, with the receiver scheduled and listening on the chosen port, for Flume to be able to push data. - -#### Configuring Flume -Configure Flume agent to send data to an Avro sink by having the following in the configuration file. - - agent.sinks = avroSink - agent.sinks.avroSink.type = avro - agent.sinks.avroSink.channel = memoryChannel - agent.sinks.avroSink.hostname = - agent.sinks.avroSink.port = - -See the [Flume's documentation](https://flume.apache.org/documentation.html) for more information about -configuring Flume agents. - -#### Configuring Spark Streaming Application -1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). - - groupId = org.apache.spark - artifactId = spark-streaming-flume_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION_SHORT}} - -2. **Programming:** In the streaming application code, import `FlumeUtils` and create input DStream as follows. - -
    -
    - import org.apache.spark.streaming.flume._ - - val flumeStream = FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]) - - See the [API docs](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$). -
    -
    - import org.apache.spark.streaming.flume.*; - - JavaReceiverInputDStream flumeStream = - FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]); - - See the [API docs](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html). -
    -
    - from pyspark.streaming.flume import FlumeUtils - - flumeStream = FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]) - - By default, the Python API will decode Flume event body as UTF8 encoded strings. You can specify your custom decoding function to decode the body byte arrays in Flume events to any arbitrary data type. - See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils). -
    -
    - - Note that the hostname should be the same as the one used by the resource manager in the - cluster (Mesos, YARN or Spark Standalone), so that resource allocation can match the names and launch - the receiver in the right machine. - -3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. - - For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). - - For Python applications which lack SBT/Maven project management, `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, - - ./bin/spark-submit --packages org.apache.spark:spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... - - Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-flume-assembly` from the - [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-flume-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. - -## Approach 2: Pull-based Approach using a Custom Sink -Instead of Flume pushing data directly to Spark Streaming, this approach runs a custom Flume sink that allows the following. - -- Flume pushes data into the sink, and the data stays buffered. -- Spark Streaming uses a [reliable Flume receiver](streaming-programming-guide.html#receiver-reliability) - and transactions to pull data from the sink. Transactions succeed only after data is received and - replicated by Spark Streaming. - -This ensures stronger reliability and -[fault-tolerance guarantees](streaming-programming-guide.html#fault-tolerance-semantics) -than the previous approach. However, this requires configuring Flume to run a custom sink. -Here are the configuration steps. - -#### General Requirements -Choose a machine that will run the custom sink in a Flume agent. The rest of the Flume pipeline is configured to send data to that agent. Machines in the Spark cluster should have access to the chosen machine running the custom sink. - -#### Configuring Flume -Configuring Flume on the chosen machine requires the following two steps. - -1. **Sink JARs**: Add the following JARs to Flume's classpath (see [Flume's documentation](https://flume.apache.org/documentation.html) to see how) in the machine designated to run the custom sink. - - (i) *Custom sink JAR*: Download the JAR corresponding to the following artifact (or [direct link](http://search.maven.org/remotecontent?filepath=org/apache/spark/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}/{{site.SPARK_VERSION_SHORT}}/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}-{{site.SPARK_VERSION_SHORT}}.jar)). - - groupId = org.apache.spark - artifactId = spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION_SHORT}} - - (ii) *Scala library JAR*: Download the Scala library JAR for Scala {{site.SCALA_VERSION}}. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/scala-lang/scala-library/{{site.SCALA_VERSION}}/scala-library-{{site.SCALA_VERSION}}.jar)). - - groupId = org.scala-lang - artifactId = scala-library - version = {{site.SCALA_VERSION}} - - (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.5/commons-lang3-3.5.jar)). - - groupId = org.apache.commons - artifactId = commons-lang3 - version = 3.5 - -2. **Configuration file**: On that machine, configure Flume agent to send data to an Avro sink by having the following in the configuration file. - - agent.sinks = spark - agent.sinks.spark.type = org.apache.spark.streaming.flume.sink.SparkSink - agent.sinks.spark.hostname = - agent.sinks.spark.port = - agent.sinks.spark.channel = memoryChannel - - Also, make sure that the upstream Flume pipeline is configured to send the data to the Flume agent running this sink. - -See the [Flume's documentation](https://flume.apache.org/documentation.html) for more information about -configuring Flume agents. - -#### Configuring Spark Streaming Application -1. **Linking:** In your SBT/Maven project definition, link your streaming application against the `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide). - -2. **Programming:** In the streaming application code, import `FlumeUtils` and create input DStream as follows. - -
    -
    - import org.apache.spark.streaming.flume._ - - val flumeStream = FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port]) -
    -
    - import org.apache.spark.streaming.flume.*; - - JavaReceiverInputDStreamflumeStream = - FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port]); -
    -
    - from pyspark.streaming.flume import FlumeUtils - - addresses = [([sink machine hostname 1], [sink port 1]), ([sink machine hostname 2], [sink port 2])] - flumeStream = FlumeUtils.createPollingStream(streamingContext, addresses) - - By default, the Python API will decode Flume event body as UTF8 encoded strings. You can specify your custom decoding function to decode the body byte arrays in Flume events to any arbitrary data type. - See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils). -
    -
    - - Note that each input DStream can be configured to receive data from multiple sinks. - -3. **Deploying:** This is same as the first approach. - - - diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 0ca0f2a8b54d..1103d5c73ff1 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -11,7 +11,7 @@ description: Spark Streaming programming guide and tutorial for Spark SPARK_VERS # Overview Spark Streaming is an extension of the core Spark API that enables scalable, high-throughput, fault-tolerant stream processing of live data streams. Data can be ingested from many sources -like Kafka, Flume, Kinesis, or TCP sockets, and can be processed using complex +like Kafka, Kinesis, or TCP sockets, and can be processed using complex algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`. Finally, processed data can be pushed out to filesystems, databases, and live dashboards. In fact, you can apply Spark's @@ -40,7 +40,7 @@ stream of results in batches. Spark Streaming provides a high-level abstraction called *discretized stream* or *DStream*, which represents a continuous stream of data. DStreams can be created either from input data -streams from sources such as Kafka, Flume, and Kinesis, or by applying high-level +streams from sources such as Kafka, and Kinesis, or by applying high-level operations on other DStreams. Internally, a DStream is represented as a sequence of [RDDs](api/scala/index.html#org.apache.spark.rdd.RDD). @@ -393,7 +393,7 @@ Similar to Spark, Spark Streaming is available through Maven Central. To write y -For ingesting data from sources like Kafka, Flume, and Kinesis that are not present in the Spark +For ingesting data from sources like Kafka and Kinesis that are not present in the Spark Streaming core API, you will have to add the corresponding artifact `spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}` to the dependencies. For example, @@ -402,7 +402,6 @@ some of the common ones are as follows. -
    SourceArtifact
    Kafka spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}
    Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} [Amazon Software License]
    @@ -577,7 +576,7 @@ Spark Streaming provides two categories of built-in streaming sources. - *Basic sources*: Sources directly available in the StreamingContext API. Examples: file systems, and socket connections. -- *Advanced sources*: Sources like Kafka, Flume, Kinesis, etc. are available through +- *Advanced sources*: Sources like Kafka, Kinesis, etc. are available through extra utility classes. These require linking against extra dependencies as discussed in the [linking](#linking) section. @@ -597,7 +596,7 @@ as well as to run the receiver(s). - When running a Spark Streaming program locally, do not use "local" or "local[1]" as the master URL. Either of these means that only one thread will be used for running tasks locally. If you are using - an input DStream based on a receiver (e.g. sockets, Kafka, Flume, etc.), then the single thread will + an input DStream based on a receiver (e.g. sockets, Kafka, etc.), then the single thread will be used to run the receiver, leaving no thread for processing the received data. Hence, when running locally, always use "local[*n*]" as the master URL, where *n* > number of receivers to run (see [Spark Properties](configuration.html#spark-properties) for information on how to set @@ -732,10 +731,10 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea {:.no_toc} Python API As of Spark {{site.SPARK_VERSION_SHORT}}, -out of these sources, Kafka, Kinesis and Flume are available in the Python API. +out of these sources, Kafka and Kinesis are available in the Python API. This category of sources require interfacing with external non-Spark libraries, some of them with -complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts +complex dependencies (e.g., Kafka). Hence, to minimize issues related to version conflicts of dependencies, the functionality to create DStreams from these sources has been moved to separate libraries that can be [linked](#linking) to explicitly when necessary. @@ -748,8 +747,6 @@ Some of these advanced sources are as follows. - **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.8.2.1 or higher. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. -- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Flume 1.6.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. - - **Kinesis:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kinesis Client Library 1.2.1. See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. ### Custom Sources @@ -766,7 +763,7 @@ Guide](streaming-custom-receivers.html) for details. {:.no_toc} There can be two kinds of data sources based on their *reliability*. Sources -(like Kafka and Flume) allow the transferred data to be acknowledged. If the system receiving +(like Kafka) allow the transferred data to be acknowledged. If the system receiving data from these *reliable* sources acknowledges the received data correctly, it can be ensured that no data will be lost due to any kind of failure. This leads to two kinds of receivers: @@ -1603,7 +1600,7 @@ operations on the same data). For window-based operations like `reduceByWindow` Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`. -For input streams that receive data over the network (such as, Kafka, Flume, sockets, etc.), the +For input streams that receive data over the network (such as, Kafka, sockets, etc.), the default persistence level is set to replicate the data to two nodes for fault-tolerance. Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in @@ -1973,7 +1970,7 @@ To run a Spark Streaming applications, you need to have the following. - *Package the application JAR* - You have to compile your streaming application into a JAR. If you are using [`spark-submit`](submitting-applications.html) to start the application, then you will not need to provide Spark and Spark Streaming in the JAR. However, - if your application uses [advanced sources](#advanced-sources) (e.g. Kafka, Flume), + if your application uses [advanced sources](#advanced-sources) (e.g. Kafka), then you will have to package the extra artifact they link to, along with their dependencies, in the JAR that is used to deploy the application. For example, an application using `KafkaUtils` will have to include `spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` and all its @@ -2060,7 +2057,7 @@ for graceful shutdown options) which ensure data that has been received is compl processed before shutdown. Then the upgraded application can be started, which will start processing from the same point where the earlier application left off. Note that this can be done only with input sources that support source-side buffering -(like Kafka, and Flume) as data needs to be buffered while the previous application was down and +(like Kafka) as data needs to be buffered while the previous application was down and the upgraded application is not yet up. And restarting from earlier checkpoint information of pre-upgrade code cannot be done. The checkpoint information essentially contains serialized Scala/Java/Python objects and trying to deserialize objects with new, @@ -2115,7 +2112,7 @@ highlights some of the most important ones. ### Level of Parallelism in Data Receiving {:.no_toc} -Receiving data over the network (like Kafka, Flume, socket, etc.) requires the data to be deserialized +Receiving data over the network (like Kafka, socket, etc.) requires the data to be deserialized and stored in Spark. If the data receiving becomes a bottleneck in the system, then consider parallelizing the data receiving. Note that each input DStream creates a single receiver (running on a worker machine) that receives a single stream of data. @@ -2475,14 +2472,12 @@ additional effort may be necessary to achieve exactly-once semantics. There are * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) * [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), - [FlumeUtils](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), [KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$), - Java docs * [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html), [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) and [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html) * [KafkaUtils](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html), - [FlumeUtils](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html), [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) - Python docs * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) and [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 73de1892977a..b6e427735e74 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1560,6 +1560,35 @@ streamingDf <- dropDuplicates(streamingDf, "guid", "eventTime") +### Policy for handling multiple watermarks +A streaming query can have multiple input streams that are unioned or joined together. +Each of the input streams can have a different threshold of late data that needs to +be tolerated for stateful operations. You specify these thresholds using +``withWatermarks("eventTime", delay)`` on each of the input streams. For example, consider +a query with stream-stream joins between `inputStream1` and `inputStream2`. + + inputStream1.withWatermark("eventTime1", "1 hour") + .join( + inputStream2.withWatermark("eventTime2", "2 hours"), + joinCondition) + +While executing the query, Structured Streaming individually tracks the maximum +event time seen in each input stream, calculates watermarks based on the corresponding delay, +and chooses a single global watermark with them to be used for stateful operations. By default, +the minimum is chosen as the global watermark because it ensures that no data is +accidentally dropped as too late if one of the streams falls behind the others +(for example, one of the streams stop receiving data due to upstream failures). In other words, +the global watermark will safely move at the pace of the slowest stream and the query output will +be delayed accordingly. + +However, in some cases, you may want to get faster results even if it means dropping data from the +slowest stream. Since Spark 2.4, you can set the multiple watermark policy to choose +the maximum value as the global watermark by setting the SQL configuration +``spark.sql.streaming.multipleWatermarkPolicy`` to ``max`` (default is ``min``). +This lets the global watermark move at the pace of the fastest stream. +However, as a side effect, data from the slower streams will be aggressively dropped. Hence, use +this configuration judiciously. + ### Arbitrary Stateful Operations Many usecases require more advanced stateful operations than aggregations. For example, in many usecases, you have to track sessions from data streams of events. For doing such sessionization, you will have to save arbitrary types of data as state, and perform arbitrary operations on the state using the data stream events in every trigger. Since Spark 2.2, this can be done using the operation `mapGroupsWithState` and the more powerful operation `flatMapGroupsWithState`. Both operations allow you to apply user-defined code on grouped Datasets to update user-defined state. For more concrete details, take a look at the API documentation ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html)) and the examples ([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java)). @@ -1799,8 +1828,16 @@ Here are the details of all the sinks in Spark. Append, Update, Complete None Depends on ForeachWriter implementation - More details in the next section + More details in the next section + + ForeachBatch Sink + Append, Update, Complete + None + Depends on the implementation + More details in the next section + + Console Sink Append, Update, Complete @@ -1989,22 +2026,214 @@ head(sql("select * from aggregates")) -##### Using Foreach -The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.1, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` -([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), -which has methods that get called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. +##### Using Foreach and ForeachBatch +The `foreach` and `foreachBatch` operations allow you to apply arbitrary operations and writing +logic on the output of a streaming query. They have slightly different use cases - while `foreach` +allows custom write logic on every row, `foreachBatch` allows arbitrary operations +and custom logic on the output of each micro-batch. Let's understand their usages in more detail. + +###### ForeachBatch +`foreachBatch(...)` allows you to specify a function that is executed on +the output data of every micro-batch of a streaming query. Since Spark 2.4, this is supported in Scala, Java and Python. +It takes two parameters: a DataFrame or Dataset that has the output data of a micro-batch and the unique ID of the micro-batch. + +
    +
    + +{% highlight scala %} +streamingDF.writeStream.foreachBatch { (batchDF: DataFrame, batchId: Long) => + // Transform and write batchDF +}.start() +{% endhighlight %} + +
    +
    + +{% highlight java %} +streamingDatasetOfString.writeStream().foreachBatch( + new VoidFunction2, Long> { + public void call(Dataset dataset, Long batchId) { + // Transform and write batchDF + } + } +).start(); +{% endhighlight %} + +
    +
    + +{% highlight python %} +def foreach_batch_function(df, epoch_id): + # Transform and write batchDF + pass + +streamingDF.writeStream.foreachBatch(foreach_batch_function).start() +{% endhighlight %} + +
    +
    +R is not yet supported. +
    +
    + +With `foreachBatch`, you can do the following. + +- **Reuse existing batch data sources** - For many storage systems, there may not be a streaming sink available yet, + but there may already exist a data writer for batch queries. Using `foreachBatch`, you can use the batch + data writers on the output of each micro-batch. +- **Write to multiple locations** - If you want to write the output of a streaming query to multiple locations, + then you can simply write the output DataFrame/Dataset multiple times. However, each attempt to write can + cause the output data to be recomputed (including possible re-reading of the input data). To avoid recomputations, + you should cache the output DataFrame/Dataset, write it to multiple locations, and then uncache it. Here is an outline. + + streamingDF.writeStream.foreachBatch { (batchDF: DataFrame, batchId: Long) => + batchDF.persist() + batchDF.write.format(...).save(...) // location 1 + batchDF.write.format(...).save(...) // location 2 + batchDF.unpersist() + } + +- **Apply additional DataFrame operations** - Many DataFrame and Dataset operations are not supported + in streaming DataFrames because Spark does not support generating incremental plans in those cases. + Using `foreachBatch`, you can apply some of these operations on each micro-batch output. However, you will have to reason about the end-to-end semantics of doing that operation yourself. + +**Note:** +- By default, `foreachBatch` provides only at-least-once write guarantees. However, you can use the + batchId provided to the function as way to deduplicate the output and get an exactly-once guarantee. +- `foreachBatch` does not work with the continuous processing mode as it fundamentally relies on the + micro-batch execution of a streaming query. If you write data in the continuous mode, use `foreach` instead. + + +###### Foreach +If `foreachBatch` is not an option (for example, corresponding batch data writer does not exist, or +continuous processing mode), then you can express you custom writer logic using `foreach`. +Specifically, you can express the data writing logic by dividing it into three methods: `open`, `process`, and `close`. +Since Spark 2.4, `foreach` is available in Scala, Java and Python. + +
    +
    + +In Scala, you have to extend the class `ForeachWriter` ([docs](api/scala/index.html#org.apache.spark.sql.ForeachWriter)). + +{% highlight scala %} +streamingDatasetOfString.writeStream.foreach( + new ForeachWriter[String] { + + def open(partitionId: Long, version: Long): Boolean = { + // Open connection + } + + def process(record: String): Unit = { + // Write string to connection + } + + def close(errorOrNull: Throwable): Unit = { + // Close the connection + } + } +).start() +{% endhighlight %} + +
    +
    + +In Java, you have to extend the class `ForeachWriter` ([docs](api/java/org/apache/spark/sql/ForeachWriter.html)). +{% highlight java %} +streamingDatasetOfString.writeStream().foreach( + new ForeachWriter[String] { + + @Override public boolean open(long partitionId, long version) { + // Open connection + } + + @Override public void process(String record) { + // Write string to connection + } + + @Override public void close(Throwable errorOrNull) { + // Close the connection + } + } +).start(); + +{% endhighlight %} + +
    +
    + +In Python, you can invoke foreach in two ways: in a function or in an object. +The function offers a simple way to express your processing logic but does not allow you to +deduplicate generated data when failures cause reprocessing of some input data. +For that situation you must specify the processing logic in an object. + +1. The function takes a row as input. + + {% highlight python %} + def process_row(row): + # Write row to storage + pass + + query = streamingDF.writeStream.foreach(process_row).start() + {% endhighlight %} + +2. The object has a process method and optional open and close methods: + + {% highlight python %} + class ForeachWriter: + def open(self, partition_id, epoch_id): + # Open connection. This method is optional in Python. + pass + + def process(self, row): + # Write row to connection. This method is NOT optional in Python. + pass + + def close(self, error): + # Close the connection. This method in optional in Python. + pass + + query = streamingDF.writeStream.foreach(ForeachWriter()).start() + {% endhighlight %} + +
    +
    +R is not yet supported. +
    +
    + + +**Execution semantics** +When the streaming query is started, Spark calls the function or the object’s methods in the following way: + +- A single copy of this object is responsible for all the data generated by a single task in a query. + In other words, one instance is responsible for processing one partition of the data generated in a distributed manner. + +- This object must be serializable, because each task will get a fresh serialized-deserialized copy + of the provided object. Hence, it is strongly recommended that any initialization for writing data + (for example. opening a connection or starting a transaction) is done after the open() method has + been called, which signifies that the task is ready to generate data. + +- The lifecycle of the methods are as follows: + + - For each partition with partition_id: -- The writer must be serializable, as it will be serialized and sent to the executors for execution. + - For each batch/epoch of streaming data with epoch_id: -- All the three methods, `open`, `process` and `close` will be called on the executors. + - Method open(partitionId, epochId) is called. -- The writer must do all the initialization (e.g. opening connections, starting a transaction, etc.) only when the `open` method is called. Be aware that, if there is any initialization in the class as soon as the object is created, then that initialization will happen in the driver (because that is where the instance is being created), which may not be what you intend. + - If open(...) returns true, for each row in the partition and batch/epoch, method process(row) is called. -- `version` and `partition` are two parameters in `open` that uniquely represent a set of rows that needs to be pushed out. `version` is a monotonically increasing id that increases with every trigger. `partition` is an id that represents a partition of the output, since the output is distributed and will be processed on multiple executors. + - Method close(error) is called with error (if any) seen while processing rows. -- `open` can use the `version` and `partition` to choose whether it needs to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If `false` is returned, then `process` will not be called on any row. For example, after a partial failure, some of the output partitions of the failed trigger may have already been committed to a database. Based on metadata stored in the database, the writer can identify partitions that have already been committed and accordingly return false to skip committing them again. +- The close() method (if it exists) is called if an open() method exists and returns successfully (irrespective of the return value), except if the JVM or Python process crashes in the middle. -- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is your responsibility to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. +- **Note:** The partitionId and epochId in the open() method can be used to deduplicate generated data + when failures cause reprocessing of some input data. This depends on the execution mode of the query. + If the streaming query is being executed in the micro-batch mode, then every partition represented + by a unique tuple (partition_id, epoch_id) is guaranteed to have the same data. + Hence, (partition_id, epoch_id) can be used to deduplicate and/or transactionally commit + data and achieve exactly-once guarantees. However, if the streaming query is being executed + in the continuous mode, then this guarantee does not hold and therefore should not be used for deduplication. #### Triggers The trigger settings of a streaming query defines the timing of streaming data processing, whether @@ -2709,6 +2938,78 @@ write.stream(aggDF, "memory", outputMode = "complete", checkpointLocation = "pat + +## Recovery Semantics after Changes in a Streaming Query +There are limitations on what changes in a streaming query are allowed between restarts from the +same checkpoint location. Here are a few kinds of changes that are either not allowed, or +the effect of the change is not well-defined. For all of them: + +- The term *allowed* means you can do the specified change but whether the semantics of its effect + is well-defined depends on the query and the change. + +- The term *not allowed* means you should not do the specified change as the restarted query is likely + to fail with unpredictable errors. `sdf` represents a streaming DataFrame/Dataset + generated with sparkSession.readStream. + +**Types of changes** + +- *Changes in the number or type (i.e. different source) of input sources*: This is not allowed. + +- *Changes in the parameters of input sources*: Whether this is allowed and whether the semantics + of the change are well-defined depends on the source and the query. Here are a few examples. + + - Addition/deletion/modification of rate limits is allowed: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "topic").option("maxOffsetsPerTrigger", ...)` + + - Changes to subscribed topics/files is generally not allowed as the results are unpredictable: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "newTopic")` + +- *Changes in the type of output sink*: Changes between a few specific combinations of sinks + are allowed. This needs to be verified on a case-by-case basis. Here are a few examples. + + - File sink to Kafka sink is allowed. Kafka will see only the new data. + + - Kafka sink to file sink is not allowed. + + - Kafka sink changed to foreach, or vice versa is allowed. + +- *Changes in the parameters of output sink*: Whether this is allowed and whether the semantics of + the change are well-defined depends on the sink and the query. Here are a few examples. + + - Changes to output directory of a file sink is not allowed: `sdf.writeStream.format("parquet").option("path", "/somePath")` to `sdf.writeStream.format("parquet").option("path", "/anotherPath")` + + - Changes to output topic is allowed: `sdf.writeStream.format("kafka").option("topic", "someTopic")` to `sdf.writeStream.format("kafka").option("topic", "anotherTopic")` + + - Changes to the user-defined foreach sink (that is, the `ForeachWriter` code) is allowed, but the semantics of the change depends on the code. + +- *Changes in projection / filter / map-like operations**: Some cases are allowed. For example: + + - Addition / deletion of filters is allowed: `sdf.selectExpr("a")` to `sdf.where(...).selectExpr("a").filter(...)`. + + - Changes in projections with same output schema is allowed: `sdf.selectExpr("stringColumn AS json").writeStream` to `sdf.selectExpr("anotherStringColumn AS json").writeStream` + + - Changes in projections with different output schema are conditionally allowed: `sdf.selectExpr("a").writeStream` to `sdf.selectExpr("b").writeStream` is allowed only if the output sink allows the schema change from `"a"` to `"b"`. + +- *Changes in stateful operations*: Some operations in streaming queries need to maintain + state data in order to continuously update the result. Structured Streaming automatically checkpoints + the state data to fault-tolerant storage (for example, HDFS, AWS S3, Azure Blob storage) and restores it after restart. + However, this assumes that the schema of the state data remains same across restarts. This means that + *any changes (that is, additions, deletions, or schema modifications) to the stateful operations of a streaming query are not allowed between restarts*. + Here is the list of stateful operations whose schema should not be changed between restarts in order to ensure state recovery: + + - *Streaming aggregation*: For example, `sdf.groupBy("a").agg(...)`. Any change in number or type of grouping keys or aggregates is not allowed. + + - *Streaming deduplication*: For example, `sdf.dropDuplicates("a")`. Any change in number or type of grouping keys or aggregates is not allowed. + + - *Stream-stream join*: For example, `sdf1.join(sdf2, ...)` (i.e. both inputs are generated with `sparkSession.readStream`). Changes + in the schema or equi-joining columns are not allowed. Changes in join type (outer or inner) not allowed. Other changes in the join condition are ill-defined. + + - *Arbitrary stateful operation*: For example, `sdf.groupByKey(...).mapGroupsWithState(...)` or `sdf.groupByKey(...).flatMapGroupsWithState(...)`. + Any change to the schema of the user-defined state and the type of timeout is not allowed. + Any change within the user-defined state-mapping function are allowed, but the semantic effect of the change depends on the user-defined logic. + If you really want to support state schema changes, then you can explicitly encode/decode your complex state data + structures into bytes using an encoding/decoding scheme that supports schema migration. For example, + if you save your state as Avro-encoded bytes, then you are free to change the Avro-state-schema between query + restarts as the binary state will always be restored successfully. + # Continuous Processing ## [Experimental] {:.no_toc} diff --git a/examples/src/main/python/streaming/flume_wordcount.py b/examples/src/main/python/streaming/flume_wordcount.py deleted file mode 100644 index c8ea92b61ca6..000000000000 --- a/examples/src/main/python/streaming/flume_wordcount.py +++ /dev/null @@ -1,56 +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. -# - -r""" - Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - Usage: flume_wordcount.py - - To run this on your local machine, you need to setup Flume first, see - https://flume.apache.org/documentation.html - - and then run the example - `$ bin/spark-submit --jars \ - external/flume-assembly/target/scala-*/spark-streaming-flume-assembly-*.jar \ - examples/src/main/python/streaming/flume_wordcount.py \ - localhost 12345 -""" -from __future__ import print_function - -import sys - -from pyspark import SparkContext -from pyspark.streaming import StreamingContext -from pyspark.streaming.flume import FlumeUtils - -if __name__ == "__main__": - if len(sys.argv) != 3: - print("Usage: flume_wordcount.py ", file=sys.stderr) - sys.exit(-1) - - sc = SparkContext(appName="PythonStreamingFlumeWordCount") - ssc = StreamingContext(sc, 1) - - hostname, port = sys.argv[1:] - kvs = FlumeUtils.createStream(ssc, hostname, int(port)) - lines = kvs.map(lambda x: x[1]) - counts = lines.flatMap(lambda line: line.split(" ")) \ - .map(lambda word: (word, 1)) \ - .reduceByKey(lambda a, b: a+b) - counts.pprint() - - ssc.start() - ssc.awaitTermination() diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml deleted file mode 100644 index 002bd6fb7f29..000000000000 --- a/external/flume-assembly/pom.xml +++ /dev/null @@ -1,167 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 3.0.0-SNAPSHOT - ../../pom.xml - - - spark-streaming-flume-assembly_2.11 - jar - Spark Project External Flume Assembly - http://spark.apache.org/ - - - provided - streaming-flume-assembly - - - - - org.apache.spark - spark-streaming-flume_${scala.binary.version} - ${project.version} - - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - - commons-codec - commons-codec - provided - - - commons-lang - commons-lang - provided - - - commons-net - commons-net - provided - - - com.google.protobuf - protobuf-java - provided - - - org.apache.avro - avro - provided - - - org.apache.avro - avro-ipc - provided - - - org.apache.avro - avro-mapred - ${avro.mapred.classifier} - provided - - - org.scala-lang - scala-library - provided - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - log4j.properties - - - - - - - - - - - - - - flume-provided - - provided - - - - - diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml deleted file mode 100644 index 168d9d3b2ae0..000000000000 --- a/external/flume-sink/pom.xml +++ /dev/null @@ -1,140 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 3.0.0-SNAPSHOT - ../../pom.xml - - - spark-streaming-flume-sink_2.11 - - streaming-flume-sink - - jar - Spark Project External Flume Sink - http://spark.apache.org/ - - - - org.apache.flume - flume-ng-sdk - - - - com.google.guava - guava - - - - org.apache.thrift - libthrift - - - - - org.apache.flume - flume-ng-core - - - com.google.guava - guava - - - org.apache.thrift - libthrift - - - - - org.scala-lang - scala-library - - - - com.google.guava - guava - test - - - - io.netty - netty - 3.4.0.Final - test - - - org.apache.spark - spark-tags_${scala.binary.version} - - - - - org.apache.spark - spark-tags_${scala.binary.version} - test-jar - test - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.avro - avro-maven-plugin - ${avro.version} - - - ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro - - - - generate-sources - - idl-protocol - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - - - - - - diff --git a/external/flume-sink/src/main/avro/sparkflume.avdl b/external/flume-sink/src/main/avro/sparkflume.avdl deleted file mode 100644 index 8806e863ac7c..000000000000 --- a/external/flume-sink/src/main/avro/sparkflume.avdl +++ /dev/null @@ -1,40 +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. - */ - -@namespace("org.apache.spark.streaming.flume.sink") - -protocol SparkFlumeProtocol { - - record SparkSinkEvent { - map headers; - bytes body; - } - - record EventBatch { - string errorMsg = ""; // If this is empty it is a valid message, else it represents an error - string sequenceNumber; - array events; - } - - EventBatch getEventBatch (int n); - - void ack (string sequenceNumber); - - void nack (string sequenceNumber); -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala deleted file mode 100644 index 09d3fe91e42c..000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala +++ /dev/null @@ -1,127 +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.streaming.flume.sink - -import org.slf4j.{Logger, LoggerFactory} - -/** - * Copy of the org.apache.spark.Logging for being used in the Spark Sink. - * The org.apache.spark.Logging is not used so that all of Spark is not brought - * in as a dependency. - */ -private[sink] trait Logging { - // Make the log field transient so that objects with Logging can - // be serialized and used on another machine - @transient private var _log: Logger = null - - // Method to get or create the logger for this object - protected def log: Logger = { - if (_log == null) { - initializeIfNecessary() - var className = this.getClass.getName - // Ignore trailing $'s in the class names for Scala objects - if (className.endsWith("$")) { - className = className.substring(0, className.length - 1) - } - _log = LoggerFactory.getLogger(className) - } - _log - } - - // Log methods that take only a String - protected def logInfo(msg: => String) { - if (log.isInfoEnabled) log.info(msg) - } - - protected def logDebug(msg: => String) { - if (log.isDebugEnabled) log.debug(msg) - } - - protected def logTrace(msg: => String) { - if (log.isTraceEnabled) log.trace(msg) - } - - protected def logWarning(msg: => String) { - if (log.isWarnEnabled) log.warn(msg) - } - - protected def logError(msg: => String) { - if (log.isErrorEnabled) log.error(msg) - } - - // Log methods that take Throwables (Exceptions/Errors) too - protected def logInfo(msg: => String, throwable: Throwable) { - if (log.isInfoEnabled) log.info(msg, throwable) - } - - protected def logDebug(msg: => String, throwable: Throwable) { - if (log.isDebugEnabled) log.debug(msg, throwable) - } - - protected def logTrace(msg: => String, throwable: Throwable) { - if (log.isTraceEnabled) log.trace(msg, throwable) - } - - protected def logWarning(msg: => String, throwable: Throwable) { - if (log.isWarnEnabled) log.warn(msg, throwable) - } - - protected def logError(msg: => String, throwable: Throwable) { - if (log.isErrorEnabled) log.error(msg, throwable) - } - - protected def isTraceEnabled(): Boolean = { - log.isTraceEnabled - } - - private def initializeIfNecessary() { - if (!Logging.initialized) { - Logging.initLock.synchronized { - if (!Logging.initialized) { - initializeLogging() - } - } - } - } - - private def initializeLogging() { - Logging.initialized = true - - // Force a call into slf4j to initialize it. Avoids this happening from multiple threads - // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html - log - } -} - -private[sink] object Logging { - @volatile private var initialized = false - val initLock = new Object() - try { - // We use reflection here to handle the case where users remove the - // slf4j-to-jul bridge order to route their logs to JUL. - // scalastyle:off classforname - val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") - // scalastyle:on classforname - bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) - val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] - if (!installed) { - bridgeClass.getMethod("install").invoke(null) - } - } catch { - case e: ClassNotFoundException => // can't log anything yet so just fail silently - } -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala deleted file mode 100644 index 8050ec357e26..000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala +++ /dev/null @@ -1,166 +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.streaming.flume.sink - -import java.util.UUID -import java.util.concurrent.{CountDownLatch, Executors} -import java.util.concurrent.atomic.AtomicLong - -import scala.collection.mutable - -import org.apache.flume.Channel - -/** - * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process - * requests. Each getEvents, ack and nack call is forwarded to an instance of this class. - * @param threads Number of threads to use to process requests. - * @param channel The channel that the sink pulls events from - * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark - * is rolled back. - */ -// Flume forces transactions to be thread-local. So each transaction *must* be committed, or -// rolled back from the thread it was originally created in. So each getEvents call from Spark -// creates a TransactionProcessor which runs in a new thread, in which the transaction is created -// and events are pulled off the channel. Once the events are sent to spark, -// that thread is blocked and the TransactionProcessor is saved in a map, -// until an ACK or NACK comes back or the transaction times out (after the specified timeout). -// When the response comes or a timeout is hit, the TransactionProcessor is retrieved and then -// unblocked, at which point the transaction is committed or rolled back. - -private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, - val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol with Logging { - val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, - new SparkSinkThreadFactory("Spark Sink Processor Thread - %d"))) - // Protected by `sequenceNumberToProcessor` - private val sequenceNumberToProcessor = mutable.HashMap[CharSequence, TransactionProcessor]() - // This sink will not persist sequence numbers and reuses them if it gets restarted. - // So it is possible to commit a transaction which may have been meant for the sink before the - // restart. - // Since the new txn may not have the same sequence number we must guard against accidentally - // committing a new transaction. To reduce the probability of that happening a random string is - // prepended to the sequence number. Does not change for life of sink - private val seqBase = UUID.randomUUID().toString.substring(0, 8) - private val seqCounter = new AtomicLong(0) - - // Protected by `sequenceNumberToProcessor` - private var stopped = false - - @volatile private var isTest = false - private var testLatch: CountDownLatch = null - - /** - * Returns a bunch of events to Spark over Avro RPC. - * @param n Maximum number of events to return in a batch - * @return [[EventBatch]] instance that has a sequence number and an array of at most n events - */ - override def getEventBatch(n: Int): EventBatch = { - logDebug("Got getEventBatch call from Spark.") - val sequenceNumber = seqBase + seqCounter.incrementAndGet() - createProcessor(sequenceNumber, n) match { - case Some(processor) => - transactionExecutorOpt.foreach(_.submit(processor)) - // Wait until a batch is available - will be an error if error message is non-empty - val batch = processor.getEventBatch - if (SparkSinkUtils.isErrorBatch(batch)) { - // Remove the processor if it is an error batch since no ACK is sent. - removeAndGetProcessor(sequenceNumber) - logWarning("Received an error batch - no events were received from channel! ") - } - batch - case None => - new EventBatch("Spark sink has been stopped!", "", java.util.Collections.emptyList()) - } - } - - private def createProcessor(seq: String, n: Int): Option[TransactionProcessor] = { - sequenceNumberToProcessor.synchronized { - if (!stopped) { - val processor = new TransactionProcessor( - channel, seq, n, transactionTimeout, backOffInterval, this) - sequenceNumberToProcessor.put(seq, processor) - if (isTest) { - processor.countDownWhenBatchAcked(testLatch) - } - Some(processor) - } else { - None - } - } - } - - /** - * Called by Spark to indicate successful commit of a batch - * @param sequenceNumber The sequence number of the event batch that was successful - */ - override def ack(sequenceNumber: CharSequence): Void = { - logDebug("Received Ack for batch with sequence number: " + sequenceNumber) - completeTransaction(sequenceNumber, success = true) - null - } - - /** - * Called by Spark to indicate failed commit of a batch - * @param sequenceNumber The sequence number of the event batch that failed - * @return - */ - override def nack(sequenceNumber: CharSequence): Void = { - completeTransaction(sequenceNumber, success = false) - logInfo("Spark failed to commit transaction. Will reattempt events.") - null - } - - /** - * Helper method to commit or rollback a transaction. - * @param sequenceNumber The sequence number of the batch that was completed - * @param success Whether the batch was successful or not. - */ - private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { - removeAndGetProcessor(sequenceNumber).foreach { processor => - processor.batchProcessed(success) - } - } - - /** - * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. - * @param sequenceNumber - * @return An `Option` of the transaction processor for the corresponding batch. Note that this - * instance is no longer tracked and the caller is responsible for that txn processor. - */ - private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): - Option[TransactionProcessor] = { - sequenceNumberToProcessor.synchronized { - sequenceNumberToProcessor.remove(sequenceNumber.toString) - } - } - - private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { - testLatch = latch - isTest = true - } - - /** - * Shuts down the executor used to process transactions. - */ - def shutdown() { - logInfo("Shutting down Spark Avro Callback Handler") - sequenceNumberToProcessor.synchronized { - stopped = true - sequenceNumberToProcessor.values.foreach(_.shutdown()) - } - transactionExecutorOpt.foreach(_.shutdownNow()) - } -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala deleted file mode 100644 index e5b63aa1a77e..000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ /dev/null @@ -1,171 +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.streaming.flume.sink - -import java.net.InetSocketAddress -import java.util.concurrent._ - -import org.apache.avro.ipc.NettyServer -import org.apache.avro.ipc.specific.SpecificResponder -import org.apache.flume.Context -import org.apache.flume.Sink.Status -import org.apache.flume.conf.{Configurable, ConfigurationException} -import org.apache.flume.sink.AbstractSink - -/** - * A sink that uses Avro RPC to run a server that can be polled by Spark's - * FlumePollingInputDStream. This sink has the following configuration parameters: - * - * hostname - The hostname to bind to. Default: 0.0.0.0 - * port - The port to bind to. (No default - mandatory) - * timeout - Time in seconds after which a transaction is rolled back, - * if an ACK is not received from Spark within that time - * threads - Number of threads to use to receive requests from Spark (Default: 10) - * - * This sink is unlike other Flume sinks in the sense that it does not push data, - * instead the process method in this sink simply blocks the SinkRunner the first time it is - * called. This sink starts up an Avro IPC server that uses the SparkFlumeProtocol. - * - * Each time a getEventBatch call comes, creates a transaction and reads events - * from the channel. When enough events are read, the events are sent to the Spark receiver and - * the thread itself is blocked and a reference to it saved off. - * - * When the ack for that batch is received, - * the thread which created the transaction is retrieved and it commits the transaction with the - * channel from the same thread it was originally created in (since Flume transactions are - * thread local). If a nack is received instead, the sink rolls back the transaction. If no ack - * is received within the specified timeout, the transaction is rolled back too. If an ack comes - * after that, it is simply ignored and the events get re-sent. - * - */ - -class SparkSink extends AbstractSink with Logging with Configurable { - - // Size of the pool to use for holding transaction processors. - private var poolSize: Integer = SparkSinkConfig.DEFAULT_THREADS - - // Timeout for each transaction. If spark does not respond in this much time, - // rollback the transaction - private var transactionTimeout = SparkSinkConfig.DEFAULT_TRANSACTION_TIMEOUT - - // Address info to bind on - private var hostname: String = SparkSinkConfig.DEFAULT_HOSTNAME - private var port: Int = 0 - - private var backOffInterval: Int = 200 - - // Handle to the server - private var serverOpt: Option[NettyServer] = None - - // The handler that handles the callback from Avro - private var handler: Option[SparkAvroCallbackHandler] = None - - // Latch that blocks off the Flume framework from wasting 1 thread. - private val blockingLatch = new CountDownLatch(1) - - override def start() { - logInfo("Starting Spark Sink: " + getName + " on port: " + port + " and interface: " + - hostname + " with " + "pool size: " + poolSize + " and transaction timeout: " + - transactionTimeout + ".") - handler = Option(new SparkAvroCallbackHandler(poolSize, getChannel, transactionTimeout, - backOffInterval)) - val responder = new SpecificResponder(classOf[SparkFlumeProtocol], handler.get) - // Using the constructor that takes specific thread-pools requires bringing in netty - // dependencies which are being excluded in the build. In practice, - // Netty dependencies are already available on the JVM as Flume would have pulled them in. - serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) - serverOpt.foreach { server => - logInfo("Starting Avro server for sink: " + getName) - server.start() - } - super.start() - } - - override def stop() { - logInfo("Stopping Spark Sink: " + getName) - handler.foreach { callbackHandler => - callbackHandler.shutdown() - } - serverOpt.foreach { server => - logInfo("Stopping Avro Server for sink: " + getName) - server.close() - server.join() - } - blockingLatch.countDown() - super.stop() - } - - override def configure(ctx: Context) { - import SparkSinkConfig._ - hostname = ctx.getString(CONF_HOSTNAME, DEFAULT_HOSTNAME) - port = Option(ctx.getInteger(CONF_PORT)). - getOrElse(throw new ConfigurationException("The port to bind to must be specified")) - poolSize = ctx.getInteger(THREADS, DEFAULT_THREADS) - transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) - backOffInterval = ctx.getInteger(CONF_BACKOFF_INTERVAL, DEFAULT_BACKOFF_INTERVAL) - logInfo("Configured Spark Sink with hostname: " + hostname + ", port: " + port + ", " + - "poolSize: " + poolSize + ", transactionTimeout: " + transactionTimeout + ", " + - "backoffInterval: " + backOffInterval) - } - - override def process(): Status = { - // This method is called in a loop by the Flume framework - block it until the sink is - // stopped to save CPU resources. The sink runner will interrupt this thread when the sink is - // being shut down. - logInfo("Blocking Sink Runner, sink will continue to run..") - blockingLatch.await() - Status.BACKOFF - } - - private[flume] def getPort(): Int = { - serverOpt - .map(_.getPort) - .getOrElse( - throw new RuntimeException("Server was not started!") - ) - } - - /** - * Pass in a [[CountDownLatch]] for testing purposes. This batch is counted down when each - * batch is received. The test can simply call await on this latch till the expected number of - * batches are received. - * @param latch - */ - private[flume] def countdownWhenBatchReceived(latch: CountDownLatch) { - handler.foreach(_.countDownWhenBatchAcked(latch)) - } -} - -/** - * Configuration parameters and their defaults. - */ -private[flume] -object SparkSinkConfig { - val THREADS = "threads" - val DEFAULT_THREADS = 10 - - val CONF_TRANSACTION_TIMEOUT = "timeout" - val DEFAULT_TRANSACTION_TIMEOUT = 60 - - val CONF_HOSTNAME = "hostname" - val DEFAULT_HOSTNAME = "0.0.0.0" - - val CONF_PORT = "port" - - val CONF_BACKOFF_INTERVAL = "backoffInterval" - val DEFAULT_BACKOFF_INTERVAL = 200 -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala deleted file mode 100644 index 845fc8debda7..000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala +++ /dev/null @@ -1,35 +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.streaming.flume.sink - -import java.util.concurrent.ThreadFactory -import java.util.concurrent.atomic.AtomicLong - -/** - * Thread factory that generates daemon threads with a specified name format. - */ -private[sink] class SparkSinkThreadFactory(nameFormat: String) extends ThreadFactory { - - private val threadId = new AtomicLong() - - override def newThread(r: Runnable): Thread = { - val t = new Thread(r, nameFormat.format(threadId.incrementAndGet())) - t.setDaemon(true) - t - } - -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala deleted file mode 100644 index 47c0e294d6b5..000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala +++ /dev/null @@ -1,28 +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.streaming.flume.sink - -private[flume] object SparkSinkUtils { - /** - * This method determines if this batch represents an error or not. - * @param batch - The batch to check - * @return - true if the batch represents an error - */ - def isErrorBatch(batch: EventBatch): Boolean = { - !batch.getErrorMsg.toString.equals("") // If there is an error message, it is an error batch. - } -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala deleted file mode 100644 index 19e736f01697..000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala +++ /dev/null @@ -1,252 +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.streaming.flume.sink - -import java.nio.ByteBuffer -import java.util -import java.util.concurrent.{Callable, CountDownLatch, TimeUnit} - -import scala.util.control.Breaks - -import org.apache.flume.{Channel, Transaction} - -// Flume forces transactions to be thread-local (horrible, I know!) -// So the sink basically spawns a new thread to pull the events out within a transaction. -// The thread fills in the event batch object that is set before the thread is scheduled. -// After filling it in, the thread waits on a condition - which is released only -// when the success message comes back for the specific sequence number for that event batch. -/** - * This class represents a transaction on the Flume channel. This class runs a separate thread - * which owns the transaction. The thread is blocked until the success call for that transaction - * comes back with an ACK or NACK. - * @param channel The channel from which to pull events - * @param seqNum The sequence number to use for the transaction. Must be unique - * @param maxBatchSize The maximum number of events to process per batch - * @param transactionTimeout Time in seconds after which a transaction must be rolled back - * without waiting for an ACK from Spark - * @param parent The parent [[SparkAvroCallbackHandler]] instance, for reporting timeouts - */ -private class TransactionProcessor(val channel: Channel, val seqNum: String, - var maxBatchSize: Int, val transactionTimeout: Int, val backOffInterval: Int, - val parent: SparkAvroCallbackHandler) extends Callable[Void] with Logging { - - // If a real batch is not returned, we always have to return an error batch. - @volatile private var eventBatch: EventBatch = new EventBatch("Unknown Error", "", - util.Collections.emptyList()) - - // Synchronization primitives - val batchGeneratedLatch = new CountDownLatch(1) - val batchAckLatch = new CountDownLatch(1) - - // Sanity check to ensure we don't loop like crazy - val totalAttemptsToRemoveFromChannel = Int.MaxValue / 2 - - // OK to use volatile, since the change would only make this true (otherwise it will be - // changed to false - we never apply a negation operation to this) - which means the transaction - // succeeded. - @volatile private var batchSuccess = false - - @volatile private var stopped = false - - @volatile private var isTest = false - - private var testLatch: CountDownLatch = null - - // The transaction that this processor would handle - var txOpt: Option[Transaction] = None - - /** - * Get an event batch from the channel. This method will block until a batch of events is - * available from the channel. If no events are available after a large number of attempts of - * polling the channel, this method will return an [[EventBatch]] with a non-empty error message - * - * @return An [[EventBatch]] instance with sequence number set to seqNum, filled with a - * maximum of maxBatchSize events - */ - def getEventBatch: EventBatch = { - batchGeneratedLatch.await() - eventBatch - } - - /** - * This method is to be called by the sink when it receives an ACK or NACK from Spark. This - * method is a no-op if it is called after transactionTimeout has expired since - * getEventBatch returned a batch of events. - * @param success True if an ACK was received and the transaction should be committed, else false. - */ - def batchProcessed(success: Boolean) { - logDebug("Batch processed for sequence number: " + seqNum) - batchSuccess = success - batchAckLatch.countDown() - } - - private[flume] def shutdown(): Unit = { - logDebug("Shutting down transaction processor") - stopped = true - } - - /** - * Populates events into the event batch. If the batch cannot be populated, - * this method will not set the events into the event batch, but it sets an error message. - */ - private def populateEvents() { - try { - txOpt = Option(channel.getTransaction) - if(txOpt.isEmpty) { - eventBatch.setErrorMsg("Something went wrong. Channel was " + - "unable to create a transaction!") - } - txOpt.foreach { tx => - tx.begin() - val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) - val loop = new Breaks - var gotEventsInThisTxn = false - var loopCounter: Int = 0 - loop.breakable { - while (!stopped && events.size() < maxBatchSize - && loopCounter < totalAttemptsToRemoveFromChannel) { - loopCounter += 1 - Option(channel.take()) match { - case Some(event) => - events.add(new SparkSinkEvent(toCharSequenceMap(event.getHeaders), - ByteBuffer.wrap(event.getBody))) - gotEventsInThisTxn = true - case None => - if (!gotEventsInThisTxn && !stopped) { - logDebug("Sleeping for " + backOffInterval + " millis as no events were read in" + - " the current transaction") - TimeUnit.MILLISECONDS.sleep(backOffInterval) - } else { - loop.break() - } - } - } - } - if (!gotEventsInThisTxn && !stopped) { - val msg = "Tried several times, " + - "but did not get any events from the channel!" - logWarning(msg) - eventBatch.setErrorMsg(msg) - } else { - // At this point, the events are available, so fill them into the event batch - eventBatch = new EventBatch("", seqNum, events) - } - } - } catch { - case interrupted: InterruptedException => - // Don't pollute logs if the InterruptedException came from this being stopped - if (!stopped) { - logWarning("Error while processing transaction.", interrupted) - } - case e: Exception => - logWarning("Error while processing transaction.", e) - eventBatch.setErrorMsg(e.getMessage) - try { - txOpt.foreach { tx => - rollbackAndClose(tx, close = true) - } - } finally { - txOpt = None - } - } finally { - batchGeneratedLatch.countDown() - } - } - - /** - * Waits for upto transactionTimeout seconds for an ACK. If an ACK comes in - * this method commits the transaction with the channel. If the ACK does not come in within - * that time or a NACK comes in, this method rolls back the transaction. - */ - private def processAckOrNack() { - batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) - txOpt.foreach { tx => - if (batchSuccess) { - try { - logDebug("Committing transaction") - tx.commit() - } catch { - case e: Exception => - logWarning("Error while attempting to commit transaction. Transaction will be rolled " + - "back", e) - rollbackAndClose(tx, close = false) // tx will be closed later anyway - } finally { - tx.close() - if (isTest) { - testLatch.countDown() - } - } - } else { - logWarning("Spark could not commit transaction, NACK received. Rolling back transaction.") - rollbackAndClose(tx, close = true) - // This might have been due to timeout or a NACK. Either way the following call does not - // cause issues. This is required to ensure the TransactionProcessor instance is not leaked - parent.removeAndGetProcessor(seqNum) - } - } - } - - /** - * Helper method to rollback and optionally close a transaction - * @param tx The transaction to rollback - * @param close Whether the transaction should be closed or not after rolling back - */ - private def rollbackAndClose(tx: Transaction, close: Boolean) { - try { - logWarning("Spark was unable to successfully process the events. Transaction is being " + - "rolled back.") - tx.rollback() - } catch { - case e: Exception => - logError("Error rolling back transaction. Rollback may have failed!", e) - } finally { - if (close) { - tx.close() - } - } - } - - /** - * Helper method to convert a Map[String, String] to Map[CharSequence, CharSequence] - * @param inMap The map to be converted - * @return The converted map - */ - private def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, - CharSequence] = { - val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) - charSeqMap.putAll(inMap) - charSeqMap - } - - /** - * When the thread is started it sets as many events as the batch size or less (if enough - * events aren't available) into the eventBatch and object and lets any threads waiting on the - * [[getEventBatch]] method to proceed. Then this thread waits for acks or nacks to come in, - * or for a specified timeout and commits or rolls back the transaction. - * @return - */ - override def call(): Void = { - populateEvents() - processAckOrNack() - null - } - - private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { - testLatch = latch - isTest = true - } -} diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties deleted file mode 100644 index 1e3f163f95c0..000000000000 --- a/external/flume-sink/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +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. -# - -# Set everything to be logged to the file streaming/target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN - diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala deleted file mode 100644 index e8ca1e716394..000000000000 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ /dev/null @@ -1,218 +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.streaming.flume.sink - -import java.net.InetSocketAddress -import java.nio.charset.StandardCharsets -import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.JavaConverters._ -import scala.concurrent.{ExecutionContext, Future} -import scala.util.{Failure, Success} - -import org.apache.avro.ipc.NettyTransceiver -import org.apache.avro.ipc.specific.SpecificRequestor -import org.apache.flume.Context -import org.apache.flume.channel.MemoryChannel -import org.apache.flume.event.EventBuilder -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory - -// Due to MNG-1378, there is not a way to include test dependencies transitively. -// We cannot include Spark core tests as a dependency here because it depends on -// Spark core main, which has too many dependencies to require here manually. -// For this reason, we continue to use FunSuite and ignore the scalastyle checks -// that fail if this is detected. -// scalastyle:off -import org.scalatest.FunSuite - -class SparkSinkSuite extends FunSuite { -// scalastyle:on - - val eventsPerBatch = 1000 - val channelCapacity = 5000 - - test("Success with ack") { - val (channel, sink, latch) = initializeChannelAndSink() - channel.start() - sink.start() - - putEvents(channel, eventsPerBatch) - - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - - val (transceiver, client) = getTransceiverAndClient(address, 1)(0) - val events = client.getEventBatch(1000) - client.ack(events.getSequenceNumber) - assert(events.getEvents.size() === 1000) - latch.await(1, TimeUnit.SECONDS) - assertChannelIsEmpty(channel) - sink.stop() - channel.stop() - transceiver.close() - } - - test("Failure with nack") { - val (channel, sink, latch) = initializeChannelAndSink() - channel.start() - sink.start() - putEvents(channel, eventsPerBatch) - - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - - val (transceiver, client) = getTransceiverAndClient(address, 1)(0) - val events = client.getEventBatch(1000) - assert(events.getEvents.size() === 1000) - client.nack(events.getSequenceNumber) - latch.await(1, TimeUnit.SECONDS) - assert(availableChannelSlots(channel) === 4000) - sink.stop() - channel.stop() - transceiver.close() - } - - test("Failure with timeout") { - val (channel, sink, latch) = initializeChannelAndSink(Map(SparkSinkConfig - .CONF_TRANSACTION_TIMEOUT -> 1.toString)) - channel.start() - sink.start() - putEvents(channel, eventsPerBatch) - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - - val (transceiver, client) = getTransceiverAndClient(address, 1)(0) - val events = client.getEventBatch(1000) - assert(events.getEvents.size() === 1000) - latch.await(1, TimeUnit.SECONDS) - assert(availableChannelSlots(channel) === 4000) - sink.stop() - channel.stop() - transceiver.close() - } - - test("Multiple consumers") { - testMultipleConsumers(failSome = false) - } - - test("Multiple consumers with some failures") { - testMultipleConsumers(failSome = true) - } - - def testMultipleConsumers(failSome: Boolean): Unit = { - implicit val executorContext = ExecutionContext - .fromExecutorService(Executors.newFixedThreadPool(5)) - val (channel, sink, latch) = initializeChannelAndSink(Map.empty, 5) - channel.start() - sink.start() - (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch)) - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - val transceiversAndClients = getTransceiverAndClient(address, 5) - val batchCounter = new CountDownLatch(5) - val counter = new AtomicInteger(0) - transceiversAndClients.foreach(x => { - Future { - val client = x._2 - val events = client.getEventBatch(1000) - if (!failSome || counter.getAndIncrement() % 2 == 0) { - client.ack(events.getSequenceNumber) - } else { - client.nack(events.getSequenceNumber) - throw new RuntimeException("Sending NACK for failure!") - } - events - }.onComplete { - case Success(events) => - assert(events.getEvents.size() === 1000) - batchCounter.countDown() - case Failure(t) => - // Don't re-throw the exception, causes a nasty unnecessary stack trace on stdout - batchCounter.countDown() - } - }) - batchCounter.await() - latch.await(1, TimeUnit.SECONDS) - executorContext.shutdown() - if(failSome) { - assert(availableChannelSlots(channel) === 3000) - } else { - assertChannelIsEmpty(channel) - } - sink.stop() - channel.stop() - transceiversAndClients.foreach(x => x._1.close()) - } - - private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty, - batchCounter: Int = 1): (MemoryChannel, SparkSink, CountDownLatch) = { - val channel = new MemoryChannel() - val channelContext = new Context() - - channelContext.put("capacity", channelCapacity.toString) - channelContext.put("transactionCapacity", 1000.toString) - channelContext.put("keep-alive", 0.toString) - channelContext.putAll(overrides.asJava) - channel.setName(scala.util.Random.nextString(10)) - channel.configure(channelContext) - - val sink = new SparkSink() - val sinkContext = new Context() - sinkContext.put(SparkSinkConfig.CONF_HOSTNAME, "0.0.0.0") - sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) - sink.configure(sinkContext) - sink.setChannel(channel) - val latch = new CountDownLatch(batchCounter) - sink.countdownWhenBatchReceived(latch) - (channel, sink, latch) - } - - private def putEvents(ch: MemoryChannel, count: Int): Unit = { - val tx = ch.getTransaction - tx.begin() - (1 to count).foreach(x => - ch.put(EventBuilder.withBody(x.toString.getBytes(StandardCharsets.UTF_8)))) - tx.commit() - tx.close() - } - - private def getTransceiverAndClient(address: InetSocketAddress, - count: Int): Seq[(NettyTransceiver, SparkFlumeProtocol.Callback)] = { - - (1 to count).map(_ => { - lazy val channelFactoryExecutor = Executors.newCachedThreadPool( - new SparkSinkThreadFactory("Flume Receiver Channel Thread - %d")) - lazy val channelFactory = - new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) - val transceiver = new NettyTransceiver(address, channelFactory) - val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) - (transceiver, client) - }) - } - - private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { - assert(availableChannelSlots(channel) === channelCapacity) - } - - private def availableChannelSlots(channel: MemoryChannel): Int = { - val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") - queueRemaining.setAccessible(true) - val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") - m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] - } -} diff --git a/external/flume/pom.xml b/external/flume/pom.xml deleted file mode 100644 index 1410ef7f4702..000000000000 --- a/external/flume/pom.xml +++ /dev/null @@ -1,89 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 3.0.0-SNAPSHOT - ../../pom.xml - - - spark-streaming-flume_2.11 - - streaming-flume - - jar - Spark Project External Flume - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-streaming-flume-sink_${scala.binary.version} - ${project.version} - - - org.apache.flume - flume-ng-core - - - org.apache.flume - flume-ng-sdk - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-tags_${scala.binary.version} - - - - - org.apache.spark - spark-tags_${scala.binary.version} - test-jar - test - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/external/flume/src/main/java/org/apache/spark/examples/JavaFlumeEventCount.java b/external/flume/src/main/java/org/apache/spark/examples/JavaFlumeEventCount.java deleted file mode 100644 index 4e3420d9c3b0..000000000000 --- a/external/flume/src/main/java/org/apache/spark/examples/JavaFlumeEventCount.java +++ /dev/null @@ -1,67 +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.examples.streaming; - -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.*; -import org.apache.spark.streaming.api.java.*; -import org.apache.spark.streaming.flume.FlumeUtils; -import org.apache.spark.streaming.flume.SparkFlumeEvent; - -/** - * Produces a count of events received from Flume. - * - * This should be used in conjunction with an AvroSink in Flume. It will start - * an Avro server on at the request host:port address and listen for requests. - * Your Flume AvroSink should be pointed to this address. - * - * Usage: JavaFlumeEventCount - * is the host the Flume receiver will be started on - a receiver - * creates a server and listens for flume events. - * is the port the Flume receiver will listen on. - * - * To run this example: - * `$ bin/run-example org.apache.spark.examples.streaming.JavaFlumeEventCount ` - */ -public final class JavaFlumeEventCount { - private JavaFlumeEventCount() { - } - - public static void main(String[] args) throws Exception { - if (args.length != 2) { - System.err.println("Usage: JavaFlumeEventCount "); - System.exit(1); - } - - String host = args[0]; - int port = Integer.parseInt(args[1]); - - Duration batchInterval = new Duration(2000); - SparkConf sparkConf = new SparkConf().setAppName("JavaFlumeEventCount"); - JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, batchInterval); - JavaReceiverInputDStream flumeStream = - FlumeUtils.createStream(ssc, host, port); - - flumeStream.count(); - - flumeStream.count().map(in -> "Received " + in + " flume events.").print(); - - ssc.start(); - ssc.awaitTermination(); - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/examples/FlumeEventCount.scala b/external/flume/src/main/scala/org/apache/spark/examples/FlumeEventCount.scala deleted file mode 100644 index f877f79391b3..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/examples/FlumeEventCount.scala +++ /dev/null @@ -1,68 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import org.apache.spark.SparkConf -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ -import org.apache.spark.streaming.flume._ -import org.apache.spark.util.IntParam - -/** - * Produces a count of events received from Flume. - * - * This should be used in conjunction with an AvroSink in Flume. It will start - * an Avro server on at the request host:port address and listen for requests. - * Your Flume AvroSink should be pointed to this address. - * - * Usage: FlumeEventCount - * is the host the Flume receiver will be started on - a receiver - * creates a server and listens for flume events. - * is the port the Flume receiver will listen on. - * - * To run this example: - * `$ bin/run-example org.apache.spark.examples.streaming.FlumeEventCount ` - */ -object FlumeEventCount { - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println( - "Usage: FlumeEventCount ") - System.exit(1) - } - - val Array(host, IntParam(port)) = args - - val batchInterval = Milliseconds(2000) - - // Create the context and set the batch size - val sparkConf = new SparkConf().setAppName("FlumeEventCount") - val ssc = new StreamingContext(sparkConf, batchInterval) - - // Create a flume stream - val stream = FlumeUtils.createStream(ssc, host, port, StorageLevel.MEMORY_ONLY_SER_2) - - // Print out the count of events received from this server in each batch - stream.count().map(cnt => "Received " + cnt + " flume events." ).print() - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/external/flume/src/main/scala/org/apache/spark/examples/FlumePollingEventCount.scala b/external/flume/src/main/scala/org/apache/spark/examples/FlumePollingEventCount.scala deleted file mode 100644 index 79a4027ca5bd..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/examples/FlumePollingEventCount.scala +++ /dev/null @@ -1,65 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import org.apache.spark.SparkConf -import org.apache.spark.streaming._ -import org.apache.spark.streaming.flume._ -import org.apache.spark.util.IntParam - -/** - * Produces a count of events received from Flume. - * - * This should be used in conjunction with the Spark Sink running in a Flume agent. See - * the Spark Streaming programming guide for more details. - * - * Usage: FlumePollingEventCount - * `host` is the host on which the Spark Sink is running. - * `port` is the port at which the Spark Sink is listening. - * - * To run this example: - * `$ bin/run-example org.apache.spark.examples.streaming.FlumePollingEventCount [host] [port] ` - */ -object FlumePollingEventCount { - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println( - "Usage: FlumePollingEventCount ") - System.exit(1) - } - - val Array(host, IntParam(port)) = args - - val batchInterval = Milliseconds(2000) - - // Create the context and set the batch size - val sparkConf = new SparkConf().setAppName("FlumePollingEventCount") - val ssc = new StreamingContext(sparkConf, batchInterval) - - // Create a flume stream that polls the Spark Sink running in a Flume agent - val stream = FlumeUtils.createPollingStream(ssc, host, port) - - // Print out the count of events received from this server in each batch - stream.count().map(cnt => "Received " + cnt + " flume events." ).print() - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala deleted file mode 100644 index 07c528647773..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala +++ /dev/null @@ -1,72 +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.streaming.flume - -import java.io.{ObjectInput, ObjectOutput} - -import scala.collection.JavaConverters._ - -import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils - -/** - * A simple object that provides the implementation of readExternal and writeExternal for both - * the wrapper classes for Flume-style Events. - */ -private[streaming] object EventTransformer extends Logging { - def readExternal(in: ObjectInput): (java.util.HashMap[CharSequence, CharSequence], - Array[Byte]) = { - val bodyLength = in.readInt() - val bodyBuff = new Array[Byte](bodyLength) - in.readFully(bodyBuff) - - val numHeaders = in.readInt() - val headers = new java.util.HashMap[CharSequence, CharSequence] - - for (i <- 0 until numHeaders) { - val keyLength = in.readInt() - val keyBuff = new Array[Byte](keyLength) - in.readFully(keyBuff) - val key: String = Utils.deserialize(keyBuff) - - val valLength = in.readInt() - val valBuff = new Array[Byte](valLength) - in.readFully(valBuff) - val value: String = Utils.deserialize(valBuff) - - headers.put(key, value) - } - (headers, bodyBuff) - } - - def writeExternal(out: ObjectOutput, headers: java.util.Map[CharSequence, CharSequence], - body: Array[Byte]) { - out.writeInt(body.length) - out.write(body) - val numHeaders = headers.size() - out.writeInt(numHeaders) - for ((k, v) <- headers.asScala) { - val keyBuff = Utils.serialize(k.toString) - out.writeInt(keyBuff.length) - out.write(keyBuff) - val valBuff = Utils.serialize(v.toString) - out.writeInt(valBuff.length) - out.write(valBuff) - } - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala deleted file mode 100644 index 8af7c2343106..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala +++ /dev/null @@ -1,166 +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.streaming.flume - -import scala.collection.mutable.ArrayBuffer - -import com.google.common.base.Throwables - -import org.apache.spark.internal.Logging -import org.apache.spark.streaming.flume.sink._ - -/** - * This class implements the core functionality of [[FlumePollingReceiver]]. When started it - * pulls data from Flume, stores it to Spark and then sends an Ack or Nack. This class should be - * run via a [[java.util.concurrent.Executor]] as this implements [[Runnable]] - * - * @param receiver The receiver that owns this instance. - */ - -private[flume] class FlumeBatchFetcher(receiver: FlumePollingReceiver) extends Runnable with - Logging { - - def run(): Unit = { - while (!receiver.isStopped()) { - val connection = receiver.getConnections.poll() - val client = connection.client - var batchReceived = false - var seq: CharSequence = null - try { - getBatch(client) match { - case Some(eventBatch) => - batchReceived = true - seq = eventBatch.getSequenceNumber - val events = toSparkFlumeEvents(eventBatch.getEvents) - if (store(events)) { - sendAck(client, seq) - } else { - sendNack(batchReceived, client, seq) - } - case None => - } - } catch { - case e: Exception => - Throwables.getRootCause(e) match { - // If the cause was an InterruptedException, then check if the receiver is stopped - - // if yes, just break out of the loop. Else send a Nack and log a warning. - // In the unlikely case, the cause was not an Exception, - // then just throw it out and exit. - case interrupted: InterruptedException => - if (!receiver.isStopped()) { - logWarning("Interrupted while receiving data from Flume", interrupted) - sendNack(batchReceived, client, seq) - } - case exception: Exception => - logWarning("Error while receiving data from Flume", exception) - sendNack(batchReceived, client, seq) - } - } finally { - receiver.getConnections.add(connection) - } - } - } - - /** - * Gets a batch of events from the specified client. This method does not handle any exceptions - * which will be propagated to the caller. - * @param client Client to get events from - * @return [[Some]] which contains the event batch if Flume sent any events back, else [[None]] - */ - private def getBatch(client: SparkFlumeProtocol.Callback): Option[EventBatch] = { - val eventBatch = client.getEventBatch(receiver.getMaxBatchSize) - if (!SparkSinkUtils.isErrorBatch(eventBatch)) { - // No error, proceed with processing data - logDebug(s"Received batch of ${eventBatch.getEvents.size} events with sequence " + - s"number: ${eventBatch.getSequenceNumber}") - Some(eventBatch) - } else { - logWarning("Did not receive events from Flume agent due to error on the Flume agent: " + - eventBatch.getErrorMsg) - None - } - } - - /** - * Store the events in the buffer to Spark. This method will not propagate any exceptions, - * but will propagate any other errors. - * @param buffer The buffer to store - * @return true if the data was stored without any exception being thrown, else false - */ - private def store(buffer: ArrayBuffer[SparkFlumeEvent]): Boolean = { - try { - receiver.store(buffer) - true - } catch { - case e: Exception => - logWarning("Error while attempting to store data received from Flume", e) - false - } - } - - /** - * Send an ack to the client for the sequence number. This method does not handle any exceptions - * which will be propagated to the caller. - * @param client client to send the ack to - * @param seq sequence number of the batch to be ack-ed. - * @return - */ - private def sendAck(client: SparkFlumeProtocol.Callback, seq: CharSequence): Unit = { - logDebug("Sending ack for sequence number: " + seq) - client.ack(seq) - logDebug("Ack sent for sequence number: " + seq) - } - - /** - * This method sends a Nack if a batch was received to the client with the given sequence - * number. Any exceptions thrown by the RPC call is simply thrown out as is - no effort is made - * to handle it. - * @param batchReceived true if a batch was received. If this is false, no nack is sent - * @param client The client to which the nack should be sent - * @param seq The sequence number of the batch that is being nack-ed. - */ - private def sendNack(batchReceived: Boolean, client: SparkFlumeProtocol.Callback, - seq: CharSequence): Unit = { - if (batchReceived) { - // Let Flume know that the events need to be pushed back into the channel. - logDebug("Sending nack for sequence number: " + seq) - client.nack(seq) // If the agent is down, even this could fail and throw - logDebug("Nack sent for sequence number: " + seq) - } - } - - /** - * Utility method to convert [[SparkSinkEvent]]s to [[SparkFlumeEvent]]s - * @param events - Events to convert to SparkFlumeEvents - * @return - The SparkFlumeEvent generated from SparkSinkEvent - */ - private def toSparkFlumeEvents(events: java.util.List[SparkSinkEvent]): - ArrayBuffer[SparkFlumeEvent] = { - // Convert each Flume event to a serializable SparkFlumeEvent - val buffer = new ArrayBuffer[SparkFlumeEvent](events.size()) - var j = 0 - while (j < events.size()) { - val event = events.get(j) - val sparkFlumeEvent = new SparkFlumeEvent() - sparkFlumeEvent.event.setBody(event.getBody) - sparkFlumeEvent.event.setHeaders(event.getHeaders) - buffer += sparkFlumeEvent - j += 1 - } - buffer - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala deleted file mode 100644 index 13aa817492f7..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ /dev/null @@ -1,208 +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.streaming.flume - -import java.io.{Externalizable, ObjectInput, ObjectOutput} -import java.net.InetSocketAddress -import java.nio.ByteBuffer -import java.util.concurrent.Executors - -import scala.collection.JavaConverters._ -import scala.reflect.ClassTag - -import org.apache.avro.ipc.NettyServer -import org.apache.avro.ipc.specific.SpecificResponder -import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol, Status} -import org.jboss.netty.channel.{ChannelPipeline, ChannelPipelineFactory, Channels} -import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory -import org.jboss.netty.handler.codec.compression._ - -import org.apache.spark.internal.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.util.Utils - -private[streaming] -class FlumeInputDStream[T: ClassTag]( - _ssc: StreamingContext, - host: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean -) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { - - override def getReceiver(): Receiver[SparkFlumeEvent] = { - new FlumeReceiver(host, port, storageLevel, enableDecompression) - } -} - -/** - * A wrapper class for AvroFlumeEvent's with a custom serialization format. - * - * This is necessary because AvroFlumeEvent uses inner data structures - * which are not serializable. - */ -class SparkFlumeEvent() extends Externalizable { - var event: AvroFlumeEvent = new AvroFlumeEvent() - - /* De-serialize from bytes. */ - def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - val bodyLength = in.readInt() - val bodyBuff = new Array[Byte](bodyLength) - in.readFully(bodyBuff) - - val numHeaders = in.readInt() - val headers = new java.util.HashMap[CharSequence, CharSequence] - - for (i <- 0 until numHeaders) { - val keyLength = in.readInt() - val keyBuff = new Array[Byte](keyLength) - in.readFully(keyBuff) - val key: String = Utils.deserialize(keyBuff) - - val valLength = in.readInt() - val valBuff = new Array[Byte](valLength) - in.readFully(valBuff) - val value: String = Utils.deserialize(valBuff) - - headers.put(key, value) - } - - event.setBody(ByteBuffer.wrap(bodyBuff)) - event.setHeaders(headers) - } - - /* Serialize to bytes. */ - def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - val body = event.getBody - out.writeInt(body.remaining()) - Utils.writeByteBuffer(body, out) - - val numHeaders = event.getHeaders.size() - out.writeInt(numHeaders) - for ((k, v) <- event.getHeaders.asScala) { - val keyBuff = Utils.serialize(k.toString) - out.writeInt(keyBuff.length) - out.write(keyBuff) - val valBuff = Utils.serialize(v.toString) - out.writeInt(valBuff.length) - out.write(valBuff) - } - } -} - -private[streaming] object SparkFlumeEvent { - def fromAvroFlumeEvent(in: AvroFlumeEvent): SparkFlumeEvent = { - val event = new SparkFlumeEvent - event.event = in - event - } -} - -/** A simple server that implements Flume's Avro protocol. */ -private[streaming] -class FlumeEventServer(receiver: FlumeReceiver) extends AvroSourceProtocol { - override def append(event: AvroFlumeEvent): Status = { - receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event)) - Status.OK - } - - override def appendBatch(events: java.util.List[AvroFlumeEvent]): Status = { - events.asScala.foreach(event => receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) - Status.OK - } -} - -/** - * A NetworkReceiver which listens for events using the - * Flume Avro interface. - */ -private[streaming] -class FlumeReceiver( - host: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean - ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { - - lazy val responder = new SpecificResponder( - classOf[AvroSourceProtocol], new FlumeEventServer(this)) - var server: NettyServer = null - - private def initServer() = { - if (enableDecompression) { - val channelFactory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), - Executors.newCachedThreadPool()) - val channelPipelineFactory = new CompressionChannelPipelineFactory() - - new NettyServer( - responder, - new InetSocketAddress(host, port), - channelFactory, - channelPipelineFactory, - null) - } else { - new NettyServer(responder, new InetSocketAddress(host, port)) - } - } - - def onStart() { - synchronized { - if (server == null) { - server = initServer() - server.start() - } else { - logWarning("Flume receiver being asked to start more then once with out close") - } - } - logInfo("Flume receiver started") - } - - def onStop() { - synchronized { - if (server != null) { - server.close() - server = null - } - } - logInfo("Flume receiver stopped") - } - - override def preferredLocation: Option[String] = Option(host) - - /** - * A Netty Pipeline factory that will decompress incoming data from - * and the Netty client and compress data going back to the client. - * - * The compression on the return is required because Flume requires - * a successful response to indicate it can remove the event/batch - * from the configured channel - */ - private[streaming] - class CompressionChannelPipelineFactory extends ChannelPipelineFactory { - def getPipeline(): ChannelPipeline = { - val pipeline = Channels.pipeline() - val encoder = new ZlibEncoder(6) - pipeline.addFirst("deflater", encoder) - pipeline.addFirst("inflater", new ZlibDecoder()) - pipeline - } - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala deleted file mode 100644 index d84e289272c6..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ /dev/null @@ -1,123 +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.streaming.flume - - -import java.net.InetSocketAddress -import java.util.concurrent.{Executors, LinkedBlockingQueue, TimeUnit} - -import scala.collection.JavaConverters._ -import scala.reflect.ClassTag - -import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.avro.ipc.NettyTransceiver -import org.apache.avro.ipc.specific.SpecificRequestor -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory - -import org.apache.spark.internal.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.flume.sink._ -import org.apache.spark.streaming.receiver.Receiver - -/** - * A `ReceiverInputDStream` that can be used to read data from several Flume agents running - * [[org.apache.spark.streaming.flume.sink.SparkSink]]s. - * @param _ssc Streaming context that will execute this input stream - * @param addresses List of addresses at which SparkSinks are listening - * @param maxBatchSize Maximum size of a batch - * @param parallelism Number of parallel connections to open - * @param storageLevel The storage level to use. - * @tparam T Class type of the object of this stream - */ -private[streaming] class FlumePollingInputDStream[T: ClassTag]( - _ssc: StreamingContext, - val addresses: Seq[InetSocketAddress], - val maxBatchSize: Int, - val parallelism: Int, - storageLevel: StorageLevel - ) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { - - override def getReceiver(): Receiver[SparkFlumeEvent] = { - new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel) - } -} - -private[streaming] class FlumePollingReceiver( - addresses: Seq[InetSocketAddress], - maxBatchSize: Int, - parallelism: Int, - storageLevel: StorageLevel - ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { - - lazy val channelFactoryExecutor = - Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). - setNameFormat("Flume Receiver Channel Thread - %d").build()) - - lazy val channelFactory = - new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) - - lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism, - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build()) - - private lazy val connections = new LinkedBlockingQueue[FlumeConnection]() - - override def onStart(): Unit = { - // Create the connections to each Flume agent. - addresses.foreach { host => - val transceiver = new NettyTransceiver(host, channelFactory) - val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) - connections.add(new FlumeConnection(transceiver, client)) - } - for (i <- 0 until parallelism) { - logInfo("Starting Flume Polling Receiver worker threads..") - // Threads that pull data from Flume. - receiverExecutor.submit(new FlumeBatchFetcher(this)) - } - } - - override def onStop(): Unit = { - logInfo("Shutting down Flume Polling Receiver") - receiverExecutor.shutdown() - // Wait upto a minute for the threads to die - if (!receiverExecutor.awaitTermination(60, TimeUnit.SECONDS)) { - receiverExecutor.shutdownNow() - } - connections.asScala.foreach(_.transceiver.close()) - channelFactory.releaseExternalResources() - } - - private[flume] def getConnections: LinkedBlockingQueue[FlumeConnection] = { - this.connections - } - - private[flume] def getMaxBatchSize: Int = { - this.maxBatchSize - } -} - -/** - * A wrapper around the transceiver and the Avro IPC API. - * @param transceiver The transceiver to use for communication with Flume - * @param client The client that the callbacks are received on. - */ -private[flume] class FlumeConnection(val transceiver: NettyTransceiver, - val client: SparkFlumeProtocol.Callback) - - - diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala deleted file mode 100644 index e8623b4766ae..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala +++ /dev/null @@ -1,117 +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.streaming.flume - -import java.net.{InetSocketAddress, ServerSocket} -import java.nio.ByteBuffer -import java.nio.charset.StandardCharsets -import java.util.{List => JList} -import java.util.Collections - -import scala.collection.JavaConverters._ - -import org.apache.avro.ipc.NettyTransceiver -import org.apache.avro.ipc.specific.SpecificRequestor -import org.apache.commons.lang3.RandomUtils -import org.apache.flume.source.avro -import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} -import org.jboss.netty.channel.ChannelPipeline -import org.jboss.netty.channel.socket.SocketChannel -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory -import org.jboss.netty.handler.codec.compression.{ZlibDecoder, ZlibEncoder} - -import org.apache.spark.SparkConf -import org.apache.spark.util.Utils - -/** - * Share codes for Scala and Python unit tests - */ -private[flume] class FlumeTestUtils { - - private var transceiver: NettyTransceiver = null - - private val testPort: Int = findFreePort() - - def getTestPort(): Int = testPort - - /** Find a free port */ - private def findFreePort(): Int = { - val candidatePort = RandomUtils.nextInt(1024, 65536) - Utils.startServiceOnPort(candidatePort, (trialPort: Int) => { - val socket = new ServerSocket(trialPort) - socket.close() - (null, trialPort) - }, new SparkConf())._2 - } - - /** Send data to the flume receiver */ - def writeInput(input: JList[String], enableCompression: Boolean): Unit = { - val testAddress = new InetSocketAddress("localhost", testPort) - - val inputEvents = input.asScala.map { item => - val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(item.getBytes(StandardCharsets.UTF_8))) - event.setHeaders(Collections.singletonMap("test", "header")) - event - } - - // if last attempted transceiver had succeeded, close it - close() - - // Create transceiver - transceiver = { - if (enableCompression) { - new NettyTransceiver(testAddress, new CompressionChannelFactory(6)) - } else { - new NettyTransceiver(testAddress) - } - } - - // Create Avro client with the transceiver - val client = SpecificRequestor.getClient(classOf[AvroSourceProtocol], transceiver) - if (client == null) { - throw new AssertionError("Cannot create client") - } - - // Send data - val status = client.appendBatch(inputEvents.asJava) - if (status != avro.Status.OK) { - throw new AssertionError("Sent events unsuccessfully") - } - } - - def close(): Unit = { - if (transceiver != null) { - transceiver.close() - transceiver = null - } - } - - /** Class to create socket channel with compression */ - private class CompressionChannelFactory(compressionLevel: Int) - extends NioClientSocketChannelFactory { - - override def newChannel(pipeline: ChannelPipeline): SocketChannel = { - val encoder = new ZlibEncoder(compressionLevel) - pipeline.addFirst("deflater", encoder) - pipeline.addFirst("inflater", new ZlibDecoder()) - super.newChannel(pipeline) - } - } - -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala deleted file mode 100644 index 707193a95770..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ /dev/null @@ -1,312 +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.streaming.flume - -import java.io.{ByteArrayOutputStream, DataOutputStream} -import java.net.InetSocketAddress -import java.util.{List => JList, Map => JMap} - -import scala.collection.JavaConverters._ - -import org.apache.spark.api.java.function.PairFunction -import org.apache.spark.api.python.PythonRDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -@deprecated("Deprecated without replacement", "2.3.0") -object FlumeUtils { - private val DEFAULT_POLLING_PARALLELISM = 5 - private val DEFAULT_POLLING_BATCH_SIZE = 1000 - - /** - * Create a input stream from a Flume source. - * @param ssc StreamingContext object - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream ( - ssc: StreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[SparkFlumeEvent] = { - createStream(ssc, hostname, port, storageLevel, false) - } - - /** - * Create a input stream from a Flume source. - * @param ssc StreamingContext object - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - * @param enableDecompression should netty server decompress input stream - */ - def createStream ( - ssc: StreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean - ): ReceiverInputDStream[SparkFlumeEvent] = { - val inputStream = new FlumeInputDStream[SparkFlumeEvent]( - ssc, hostname, port, storageLevel, enableDecompression) - - inputStream - } - - /** - * Creates a input stream from a Flume source. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - */ - def createStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createStream(jssc.ssc, hostname, port) - } - - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createStream(jssc.ssc, hostname, port, storageLevel, false) - } - - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - * @param enableDecompression should netty server decompress input stream - */ - def createStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createStream(jssc.ssc, hostname, port, storageLevel, enableDecompression) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param hostname Address of the host on which the Spark Sink is running - * @param port Port of the host at which the Spark Sink is listening - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - ssc: StreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(ssc, Seq(new InetSocketAddress(hostname, port)), storageLevel) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param addresses List of InetSocketAddresses representing the hosts to connect to. - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - ssc: StreamingContext, - addresses: Seq[InetSocketAddress], - storageLevel: StorageLevel - ): ReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(ssc, addresses, storageLevel, - DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * @param addresses List of InetSocketAddresses representing the hosts to connect to. - * @param maxBatchSize Maximum number of events to be pulled from the Spark sink in a - * single RPC call - * @param parallelism Number of concurrent requests this stream should send to the sink. Note - * that having a higher number of requests concurrently being pulled will - * result in this stream using more threads - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - ssc: StreamingContext, - addresses: Seq[InetSocketAddress], - storageLevel: StorageLevel, - maxBatchSize: Int, - parallelism: Int - ): ReceiverInputDStream[SparkFlumeEvent] = { - new FlumePollingInputDStream[SparkFlumeEvent](ssc, addresses, maxBatchSize, - parallelism, storageLevel) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param hostname Hostname of the host on which the Spark Sink is running - * @param port Port of the host at which the Spark Sink is listening - */ - def createPollingStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(jssc, hostname, port, StorageLevel.MEMORY_AND_DISK_SER_2) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param hostname Hostname of the host on which the Spark Sink is running - * @param port Port of the host at which the Spark Sink is listening - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(jssc, Array(new InetSocketAddress(hostname, port)), storageLevel) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param addresses List of InetSocketAddresses on which the Spark Sink is running. - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - jssc: JavaStreamingContext, - addresses: Array[InetSocketAddress], - storageLevel: StorageLevel - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(jssc, addresses, storageLevel, - DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * @param addresses List of InetSocketAddresses on which the Spark Sink is running - * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a - * single RPC call - * @param parallelism Number of concurrent requests this stream should send to the sink. Note - * that having a higher number of requests concurrently being pulled will - * result in this stream using more threads - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - jssc: JavaStreamingContext, - addresses: Array[InetSocketAddress], - storageLevel: StorageLevel, - maxBatchSize: Int, - parallelism: Int - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) - } -} - -/** - * This is a helper class that wraps the methods in FlumeUtils into more Python-friendly class and - * function so that it can be easily instantiated and called from Python's FlumeUtils. - */ -private[flume] class FlumeUtilsPythonHelper { - - def createStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean - ): JavaPairDStream[Array[Byte], Array[Byte]] = { - val dstream = FlumeUtils.createStream(jssc, hostname, port, storageLevel, enableDecompression) - FlumeUtilsPythonHelper.toByteArrayPairDStream(dstream) - } - - def createPollingStream( - jssc: JavaStreamingContext, - hosts: JList[String], - ports: JList[Int], - storageLevel: StorageLevel, - maxBatchSize: Int, - parallelism: Int - ): JavaPairDStream[Array[Byte], Array[Byte]] = { - assert(hosts.size() == ports.size()) - val addresses = hosts.asScala.zip(ports.asScala).map { - case (host, port) => new InetSocketAddress(host, port) - } - val dstream = FlumeUtils.createPollingStream( - jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) - FlumeUtilsPythonHelper.toByteArrayPairDStream(dstream) - } - -} - -private object FlumeUtilsPythonHelper { - - private def stringMapToByteArray(map: JMap[CharSequence, CharSequence]): Array[Byte] = { - val byteStream = new ByteArrayOutputStream() - val output = new DataOutputStream(byteStream) - try { - output.writeInt(map.size) - map.asScala.foreach { kv => - PythonRDD.writeUTF(kv._1.toString, output) - PythonRDD.writeUTF(kv._2.toString, output) - } - byteStream.toByteArray - } - finally { - output.close() - } - } - - private def toByteArrayPairDStream(dstream: JavaReceiverInputDStream[SparkFlumeEvent]): - JavaPairDStream[Array[Byte], Array[Byte]] = { - dstream.mapToPair(new PairFunction[SparkFlumeEvent, Array[Byte], Array[Byte]] { - override def call(sparkEvent: SparkFlumeEvent): (Array[Byte], Array[Byte]) = { - val event = sparkEvent.event - val byteBuffer = event.getBody - val body = new Array[Byte](byteBuffer.remaining()) - byteBuffer.get(body) - (stringMapToByteArray(event.getHeaders), body) - } - }) - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala deleted file mode 100644 index a3e784a4f32e..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala +++ /dev/null @@ -1,209 +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.streaming.flume - -import java.nio.charset.StandardCharsets -import java.util.{Collections, List => JList, Map => JMap} -import java.util.concurrent._ - -import scala.collection.mutable.ArrayBuffer - -import org.apache.flume.Context -import org.apache.flume.channel.MemoryChannel -import org.apache.flume.conf.Configurables -import org.apache.flume.event.EventBuilder - -import org.apache.spark.streaming.flume.sink.{SparkSink, SparkSinkConfig} - -/** - * Share codes for Scala and Python unit tests - */ -private[flume] class PollingFlumeTestUtils { - - private val batchCount = 5 - val eventsPerBatch = 100 - private val totalEventsPerChannel = batchCount * eventsPerBatch - private val channelCapacity = 5000 - - def getTotalEvents: Int = totalEventsPerChannel * channels.size - - private val channels = new ArrayBuffer[MemoryChannel] - private val sinks = new ArrayBuffer[SparkSink] - - /** - * Start a sink and return the port of this sink - */ - def startSingleSink(): Int = { - channels.clear() - sinks.clear() - - // Start the channel and sink. - val context = new Context() - context.put("capacity", channelCapacity.toString) - context.put("transactionCapacity", "1000") - context.put("keep-alive", "0") - val channel = new MemoryChannel() - Configurables.configure(channel, context) - - val sink = new SparkSink() - context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) - Configurables.configure(sink, context) - sink.setChannel(channel) - sink.start() - - channels += (channel) - sinks += sink - - sink.getPort() - } - - /** - * Start 2 sinks and return the ports - */ - def startMultipleSinks(): Seq[Int] = { - channels.clear() - sinks.clear() - - // Start the channel and sink. - val context = new Context() - context.put("capacity", channelCapacity.toString) - context.put("transactionCapacity", "1000") - context.put("keep-alive", "0") - val channel = new MemoryChannel() - Configurables.configure(channel, context) - - val channel2 = new MemoryChannel() - Configurables.configure(channel2, context) - - val sink = new SparkSink() - context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) - Configurables.configure(sink, context) - sink.setChannel(channel) - sink.start() - - val sink2 = new SparkSink() - context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) - Configurables.configure(sink2, context) - sink2.setChannel(channel2) - sink2.start() - - sinks += sink - sinks += sink2 - channels += channel - channels += channel2 - - sinks.map(_.getPort()) - } - - /** - * Send data and wait until all data has been received - */ - def sendDataAndEnsureAllDataHasBeenReceived(): Unit = { - val executor = Executors.newCachedThreadPool() - val executorCompletion = new ExecutorCompletionService[Void](executor) - - val latch = new CountDownLatch(batchCount * channels.size) - sinks.foreach(_.countdownWhenBatchReceived(latch)) - - channels.foreach { channel => - executorCompletion.submit(new TxnSubmitter(channel)) - } - - for (i <- 0 until channels.size) { - executorCompletion.take() - } - - latch.await(15, TimeUnit.SECONDS) // Ensure all data has been received. - } - - /** - * A Python-friendly method to assert the output - */ - def assertOutput( - outputHeaders: JList[JMap[String, String]], outputBodies: JList[String]): Unit = { - require(outputHeaders.size == outputBodies.size) - val eventSize = outputHeaders.size - if (eventSize != totalEventsPerChannel * channels.size) { - throw new AssertionError( - s"Expected ${totalEventsPerChannel * channels.size} events, but was $eventSize") - } - var counter = 0 - for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) { - val eventBodyToVerify = s"${channels(k).getName}-$i" - val eventHeaderToVerify: JMap[String, String] = Collections.singletonMap(s"test-$i", "header") - var found = false - var j = 0 - while (j < eventSize && !found) { - if (eventBodyToVerify == outputBodies.get(j) && - eventHeaderToVerify == outputHeaders.get(j)) { - found = true - counter += 1 - } - j += 1 - } - } - if (counter != totalEventsPerChannel * channels.size) { - throw new AssertionError( - s"111 Expected ${totalEventsPerChannel * channels.size} events, but was $counter") - } - } - - def assertChannelsAreEmpty(): Unit = { - channels.foreach(assertChannelIsEmpty) - } - - private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { - val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") - queueRemaining.setAccessible(true) - val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") - if (m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] != channelCapacity) { - throw new AssertionError(s"Channel ${channel.getName} is not empty") - } - } - - def close(): Unit = { - sinks.foreach(_.stop()) - sinks.clear() - channels.foreach(_.stop()) - channels.clear() - } - - private class TxnSubmitter(channel: MemoryChannel) extends Callable[Void] { - override def call(): Void = { - var t = 0 - for (i <- 0 until batchCount) { - val tx = channel.getTransaction - tx.begin() - for (j <- 0 until eventsPerBatch) { - channel.put(EventBuilder.withBody( - s"${channel.getName}-$t".getBytes(StandardCharsets.UTF_8), - Collections.singletonMap(s"test-$t", "header"))) - t += 1 - } - tx.commit() - tx.close() - Thread.sleep(500) // Allow some time for the events to reach - } - null - } - } - -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java deleted file mode 100644 index 4a5da226aded..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java +++ /dev/null @@ -1,21 +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. - */ - -/** - * Spark streaming receiver for Flume. - */ -package org.apache.spark.streaming.flume; diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala deleted file mode 100644 index 9bfab68c4b8b..000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala +++ /dev/null @@ -1,23 +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.streaming - -/** - * Spark streaming receiver for Flume. - */ -package object flume diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java deleted file mode 100644 index cfedb5a042a3..000000000000 --- a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ /dev/null @@ -1,44 +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.streaming; - -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.After; -import org.junit.Before; - -public abstract class LocalJavaStreamingContext { - - protected transient JavaStreamingContext ssc; - - @Before - public void setUp() { - SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - ssc = new JavaStreamingContext(conf, new Duration(1000)); - ssc.checkpoint("checkpoint"); - } - - @After - public void tearDown() { - ssc.stop(); - ssc = null; - } -} diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java deleted file mode 100644 index 79c5b91654b4..000000000000 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java +++ /dev/null @@ -1,44 +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.streaming.flume; - -import java.net.InetSocketAddress; - -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.LocalJavaStreamingContext; - -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; -import org.junit.Test; - -public class JavaFlumePollingStreamSuite extends LocalJavaStreamingContext { - @Test - public void testFlumeStream() { - // tests the API, does not actually test data receiving - InetSocketAddress[] addresses = new InetSocketAddress[] { - new InetSocketAddress("localhost", 12345) - }; - JavaReceiverInputDStream test1 = - FlumeUtils.createPollingStream(ssc, "localhost", 12345); - JavaReceiverInputDStream test2 = FlumeUtils.createPollingStream( - ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test3 = FlumeUtils.createPollingStream( - ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test4 = FlumeUtils.createPollingStream( - ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2(), 100, 5); - } -} diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java deleted file mode 100644 index ada05f203b6a..000000000000 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java +++ /dev/null @@ -1,37 +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.streaming.flume; - -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.LocalJavaStreamingContext; - -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; -import org.junit.Test; - -public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { - @Test - public void testFlumeStream() { - // tests the API, does not actually test data receiving - JavaReceiverInputDStream test1 = FlumeUtils.createStream(ssc, "localhost", - 12345); - JavaReceiverInputDStream test2 = FlumeUtils.createStream(ssc, "localhost", - 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test3 = FlumeUtils.createStream(ssc, "localhost", - 12345, StorageLevel.MEMORY_AND_DISK_SER_2(), false); - } -} diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties deleted file mode 100644 index fd51f8faf56b..000000000000 --- a/external/flume/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +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. -# - -# Set everything to be logged to the file target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN - diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala deleted file mode 100644 index c97a27ca7c7a..000000000000 --- a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala +++ /dev/null @@ -1,48 +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.streaming - -import java.io.{IOException, ObjectInputStream} -import java.util.concurrent.ConcurrentLinkedQueue - -import scala.reflect.ClassTag - -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.dstream.{DStream, ForEachDStream} -import org.apache.spark.util.Utils - -/** - * This is a output stream just for the testsuites. All the output is collected into a - * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. - * - * The buffer contains a sequence of RDD's, each containing a sequence of items - */ -class TestOutputStream[T: ClassTag](parent: DStream[T], - val output: ConcurrentLinkedQueue[Seq[T]] = new ConcurrentLinkedQueue[Seq[T]]()) - extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { - val collected = rdd.collect() - output.add(collected) - }, false) { - - // This is to clear the output buffer every it is read from a checkpoint - @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { - ois.defaultReadObject() - output.clear() - } -} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala deleted file mode 100644 index 9241b13c100f..000000000000 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ /dev/null @@ -1,149 +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.streaming.flume - -import java.net.InetSocketAddress -import java.util.concurrent.ConcurrentLinkedQueue - -import scala.collection.JavaConverters._ -import scala.concurrent.duration._ -import scala.language.postfixOps - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.concurrent.Eventually._ - -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.internal.Logging -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext, TestOutputStream} -import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.util.{ManualClock, Utils} - -class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { - - val maxAttempts = 5 - val batchDuration = Seconds(1) - - @transient private var _sc: SparkContext = _ - - val conf = new SparkConf() - .setMaster("local[2]") - .setAppName(this.getClass.getSimpleName) - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock") - - val utils = new PollingFlumeTestUtils - - override def beforeAll(): Unit = { - super.beforeAll() - _sc = new SparkContext(conf) - } - - override def afterAll(): Unit = { - try { - if (_sc != null) { - _sc.stop() - _sc = null - } - } finally { - super.afterAll() - } - } - - test("flume polling test") { - testMultipleTimes(() => testFlumePolling()) - } - - test("flume polling test multiple hosts") { - testMultipleTimes(() => testFlumePollingMultipleHost()) - } - - /** - * Run the given test until no more java.net.BindException's are thrown. - * Do this only up to a certain attempt limit. - */ - private def testMultipleTimes(test: () => Unit): Unit = { - var testPassed = false - var attempt = 0 - while (!testPassed && attempt < maxAttempts) { - try { - test() - testPassed = true - } catch { - case e: Exception if Utils.isBindCollision(e) => - logWarning("Exception when running flume polling test: " + e) - attempt += 1 - } - } - assert(testPassed, s"Test failed after $attempt attempts!") - } - - private def testFlumePolling(): Unit = { - try { - val port = utils.startSingleSink() - - writeAndVerify(Seq(port)) - utils.assertChannelsAreEmpty() - } finally { - utils.close() - } - } - - private def testFlumePollingMultipleHost(): Unit = { - try { - val ports = utils.startMultipleSinks() - writeAndVerify(ports) - utils.assertChannelsAreEmpty() - } finally { - utils.close() - } - } - - def writeAndVerify(sinkPorts: Seq[Int]): Unit = { - // Set up the streaming context and input streams - val ssc = new StreamingContext(_sc, batchDuration) - val addresses = sinkPorts.map(port => new InetSocketAddress("localhost", port)) - val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, - utils.eventsPerBatch, 5) - val outputQueue = new ConcurrentLinkedQueue[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputQueue) - outputStream.register() - - ssc.start() - try { - utils.sendDataAndEnsureAllDataHasBeenReceived() - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - clock.advance(batchDuration.milliseconds) - - // The eventually is required to ensure that all data in the batch has been processed. - eventually(timeout(10 seconds), interval(100 milliseconds)) { - val flattenOutput = outputQueue.asScala.toSeq.flatten - val headers = flattenOutput.map(_.event.getHeaders.asScala.map { - case (key, value) => (key.toString, value.toString) - }).map(_.asJava) - val bodies = flattenOutput.map(e => JavaUtils.bytesToString(e.event.getBody)) - utils.assertOutput(headers.asJava, bodies.asJava) - } - } finally { - // here stop ssc only, but not underlying sparkcontext - ssc.stop(false) - } - } - -} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala deleted file mode 100644 index 7bac1cc4b0ae..000000000000 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ /dev/null @@ -1,103 +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.streaming.flume - -import java.util.concurrent.ConcurrentLinkedQueue - -import scala.collection.JavaConverters._ -import scala.concurrent.duration._ -import scala.language.postfixOps - -import org.jboss.netty.channel.ChannelPipeline -import org.jboss.netty.channel.socket.SocketChannel -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory -import org.jboss.netty.handler.codec.compression._ -import org.scalatest.{BeforeAndAfter, Matchers} -import org.scalatest.concurrent.Eventually._ - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.Logging -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Milliseconds, StreamingContext, TestOutputStream} - -class FlumeStreamSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { - val conf = new SparkConf().setMaster("local[4]").setAppName("FlumeStreamSuite") - var ssc: StreamingContext = null - - test("flume input stream") { - testFlumeStream(testCompression = false) - } - - test("flume input compressed stream") { - testFlumeStream(testCompression = true) - } - - /** Run test on flume stream */ - private def testFlumeStream(testCompression: Boolean): Unit = { - val input = (1 to 100).map { _.toString } - val utils = new FlumeTestUtils - try { - val outputQueue = startContext(utils.getTestPort(), testCompression) - - eventually(timeout(10 seconds), interval(100 milliseconds)) { - utils.writeInput(input.asJava, testCompression) - } - - eventually(timeout(10 seconds), interval(100 milliseconds)) { - val outputEvents = outputQueue.asScala.toSeq.flatten.map { _.event } - outputEvents.foreach { - event => - event.getHeaders.get("test") should be("header") - } - val output = outputEvents.map(event => JavaUtils.bytesToString(event.getBody)) - output should be (input) - } - } finally { - if (ssc != null) { - ssc.stop() - } - utils.close() - } - } - - /** Setup and start the streaming context */ - private def startContext( - testPort: Int, testCompression: Boolean): (ConcurrentLinkedQueue[Seq[SparkFlumeEvent]]) = { - ssc = new StreamingContext(conf, Milliseconds(200)) - val flumeStream = FlumeUtils.createStream( - ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK, testCompression) - val outputQueue = new ConcurrentLinkedQueue[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputQueue) - outputStream.register() - ssc.start() - outputQueue - } - - /** Class to create socket channel with compression */ - private class CompressionChannelFactory(compressionLevel: Int) - extends NioClientSocketChannelFactory { - - override def newChannel(pipeline: ChannelPipeline): SocketChannel = { - val encoder = new ZlibEncoder(compressionLevel) - pipeline.addFirst("deflater", encoder) - pipeline.addFirst("inflater", new ZlibDecoder()) - super.newChannel(pipeline) - } - } -} 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 39c2cde7de40..5ee76990b54f 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 @@ -33,7 +33,7 @@ import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql.{ForeachWriter, SparkSession} +import org.apache.spark.sql.{Dataset, ForeachWriter, SparkSession} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.streaming._ @@ -900,7 +900,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { } testUtils.waitUntilOffsetAppears(topicPartition, 5) - val q = ds.writeStream.foreachBatch { (ds, epochId) => + val q = ds.writeStream.foreachBatch { (ds: Dataset[String], epochId: Long) => if (epochId == 0) { // Send more message before the tasks of the current batch start reading the current batch // data, so that the executors will prefetch messages in the next batch and drop them. In diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 81832fbdcd7e..d46c4139011d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -427,6 +427,7 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { .format("kafka") .option("checkpointLocation", checkpointDir.getCanonicalPath) .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.max.block.ms", "5000") .queryName("kafkaStream") withTopic.foreach(stream.option("topic", _)) withOutputMode.foreach(stream.outputMode(_)) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index 0acc9b8d2a0c..ba4009ef0885 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -108,7 +108,6 @@ private[spark] class DirectKafkaInputDStream[K, V]( } } - // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") private[streaming] override def name: String = s"Kafka 0.10 direct stream [$id]" protected[streaming] override val checkpointData = diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 9297c39d170c..2ec771e97714 100644 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -67,7 +67,6 @@ class DirectKafkaInputDStream[ val maxRetries = context.sparkContext.getConf.getInt( "spark.streaming.kafka.maxRetries", 1) - // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") private[streaming] override def name: String = s"Kafka direct stream [$id]" protected[streaming] override val checkpointData = diff --git a/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala index 0cd795f63887..4fb9f2f84908 100644 --- a/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala +++ b/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala @@ -17,7 +17,7 @@ package org.apache.spark.metrics.sink -import java.util.Properties +import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit import com.codahale.metrics.MetricRegistry @@ -64,11 +64,12 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val ttl = propertyToOption(GANGLIA_KEY_TTL).map(_.toInt).getOrElse(GANGLIA_DEFAULT_TTL) val dmax = propertyToOption(GANGLIA_KEY_DMAX).map(_.toInt).getOrElse(GANGLIA_DEFAULT_DMAX) val mode: UDPAddressingMode = propertyToOption(GANGLIA_KEY_MODE) - .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase)).getOrElse(GANGLIA_DEFAULT_MODE) + .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase(Locale.ROOT))) + .getOrElse(GANGLIA_DEFAULT_MODE) val pollPeriod = propertyToOption(GANGLIA_KEY_PERIOD).map(_.toInt) .getOrElse(GANGLIA_DEFAULT_PERIOD) val pollUnit: TimeUnit = propertyToOption(GANGLIA_KEY_UNIT) - .map(u => TimeUnit.valueOf(u.toUpperCase)) + .map(u => TimeUnit.valueOf(u.toUpperCase(Locale.ROOT))) .getOrElse(GANGLIA_DEFAULT_UNIT) MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index d48162007e67..3182ab15db5f 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -166,45 +166,34 @@ httpcore ${hadoop.deps.scope} + + org.apache.hadoop + hadoop-azure + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.jackson.core + jackson-core + + + com.google.guava + guava + + + - - hadoop-2.7 - - - - - - org.apache.hadoop - hadoop-azure - ${hadoop.version} - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-common - - - org.codehaus.jackson - jackson-mapper-asl - - - com.fasterxml.jackson.core - jackson-core - - - com.google.guava - guava - - - - - - 1.2.1.spark2 @@ -131,7 +130,7 @@ 1.2.1 10.12.1.1 1.10.0 - 1.5.2 + 1.5.3 nohive 1.6.0 9.3.24.v20180605 @@ -212,7 +211,6 @@ during compilation if the dependency is transivite (e.g. "graphx/" depending on "core/" and needing Hadoop classes in the classpath to compile). --> - compile compile compile compile @@ -540,7 +538,7 @@ org.lz4 lz4-java - 1.4.0 + 1.5.0 com.github.luben @@ -1805,46 +1803,6 @@ ${hive.parquet.version} compile - - org.apache.flume - flume-ng-core - ${flume.version} - ${flume.deps.scope} - - - io.netty - netty - - - org.apache.flume - flume-ng-auth - - - org.apache.thrift - libthrift - - - org.mortbay.jetty - servlet-api - - - - - org.apache.flume - flume-ng-sdk - ${flume.version} - ${flume.deps.scope} - - - io.netty - netty - - - org.apache.thrift - libthrift - - - org.apache.calcite calcite-core @@ -2635,15 +2593,6 @@ - - flume - - external/flume - external/flume-sink - external/flume-assembly - - - spark-ganglia-lgpl @@ -2674,17 +2623,9 @@ http://hadoop.apache.org/docs/ra.b.c/hadoop-project-dist/hadoop-common/dependency-analysis.html --> - - hadoop-2.6 - - - hadoop-2.7 - - 2.7.3 - 2.7.1 - + @@ -2843,9 +2784,6 @@ maven does not complain when they're provided on the command line for a sub-module that does not have them. --> - - flume-provided - hadoop-provided diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a93173803246..bf85fe0b4512 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,7 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.io.SnappyCompressionCodec.version") ) // Exclude rules for 2.4.x @@ -103,13 +104,6 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.defaultParamMap"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.org$apache$spark$ml$param$Params$_setter_$defaultParamMap_="), - // [SPARK-14681][ML] Provide label/impurity stats for spark.ml decision tree nodes - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.LeafNode"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.InternalNode"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.Node"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.this"), - // [SPARK-7132][ML] Add fit with validation set to spark.ml GBT ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a5ed9088eaa4..8b01b9079e6d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -55,16 +55,14 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects val optionallyEnabledProjects@Seq(kubernetes, mesos, yarn, - streamingFlumeSink, streamingFlume, streamingKafka, sparkGangliaLgpl, streamingKinesisAsl, dockerIntegrationTests, hadoopCloud, kubernetesIntegrationTests) = Seq("kubernetes", "mesos", "yarn", - "streaming-flume-sink", "streaming-flume", "streaming-kafka-0-8", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests", "hadoop-cloud", "kubernetes-integration-tests").map(ProjectRef(buildLocation, _)) - val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) = - Seq("network-yarn", "streaming-flume-assembly", "streaming-kafka-0-8-assembly", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") + val assemblyProjects@Seq(networkYarn, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) = + Seq("network-yarn", "streaming-kafka-0-8-assembly", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") .map(ProjectRef(buildLocation, _)) val copyJarsProjects@Seq(assembly, examples) = Seq("assembly", "examples") @@ -373,8 +371,6 @@ object SparkBuild extends PomBuild { /* Hive console settings */ enable(Hive.settings)(hive) - enable(Flume.settings)(streamingFlumeSink) - // SPARK-14738 - Remove docker tests from main Spark build // enable(DockerIntegrationTests.settings)(dockerIntegrationTests) @@ -452,9 +448,6 @@ object Unsafe { ) } -object Flume { - lazy val settings = sbtavro.SbtAvro.avroSettings -} object DockerIntegrationTests { // This serves to override the override specified in DependencyOverrides: @@ -587,8 +580,7 @@ object Assembly { .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) }, jarName in assembly := { - if (moduleName.value.contains("streaming-flume-assembly") - || moduleName.value.contains("streaming-kafka-0-8-assembly") + if (moduleName.value.contains("streaming-kafka-0-8-assembly") || moduleName.value.contains("streaming-kafka-0-10-assembly") || moduleName.value.contains("streaming-kinesis-asl-assembly")) { // This must match the same name used in maven (see external/kafka-0-8-assembly/pom.xml) @@ -694,10 +686,10 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, kubernetes, + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, yarn, tags, streamingKafka010, sqlKafka010, avro), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, kubernetes, + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, yarn, tags, streamingKafka010, sqlKafka010, avro), unidocAllClasspaths in (ScalaUnidoc, unidoc) := { diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst index 25ceabac0a54..9c256284ad69 100644 --- a/python/docs/pyspark.streaming.rst +++ b/python/docs/pyspark.streaming.rst @@ -22,10 +22,3 @@ pyspark.streaming.kinesis module :members: :undoc-members: :show-inheritance: - -pyspark.streaming.flume.module ------------------------------- -.. automodule:: pyspark.streaming.flume - :members: - :undoc-members: - :show-inheritance: diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 30ad04297c68..00ec094e7e3b 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -109,10 +109,14 @@ def _deserialize_accumulator(aid, zero_value, accum_param): from pyspark.accumulators import _accumulatorRegistry - accum = Accumulator(aid, zero_value, accum_param) - accum._deserialized = True - _accumulatorRegistry[aid] = accum - return accum + # If this certain accumulator was deserialized, don't overwrite it. + if aid in _accumulatorRegistry: + return _accumulatorRegistry[aid] + else: + accum = Accumulator(aid, zero_value, accum_param) + accum._deserialized = True + _accumulatorRegistry[aid] = accum + return accum class Accumulator(object): diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 3128d5792eea..5425d311f8c7 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1691,18 +1691,32 @@ def repeat(col, n): @since(1.5) @ignore_unicode_prefix -def split(str, pattern): +def split(str, pattern, limit=-1): """ - Splits str around pattern (pattern is a regular expression). + Splits str around matches of the given pattern. - .. note:: pattern is a string represent the regular expression. + :param str: a string expression to split + :param pattern: a string representing a regular expression. The regex string should be + a Java regular expression. + :param limit: an integer which controls the number of times `pattern` is applied. - >>> df = spark.createDataFrame([('ab12cd',)], ['s',]) - >>> df.select(split(df.s, '[0-9]+').alias('s')).collect() - [Row(s=[u'ab', u'cd'])] + * ``limit > 0``: The resulting array's length will not be more than `limit`, and the + resulting array's last entry will contain all input beyond the last + matched pattern. + * ``limit <= 0``: `pattern` will be applied as many times as possible, and the resulting + array can be of any size. + + .. versionchanged:: 3.0 + `split` now takes an optional `limit` field. If not provided, default limit value is -1. + + >>> df = spark.createDataFrame([('oneAtwoBthreeC',)], ['s',]) + >>> df.select(split(df.s, '[ABC]', 2).alias('s')).collect() + [Row(s=[u'one', u'twoBthreeC'])] + >>> df.select(split(df.s, '[ABC]', -1).alias('s')).collect() + [Row(s=[u'one', u'two', u'three', u''])] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.split(_to_java_column(str), pattern)) + return Column(sc._jvm.functions.split(_to_java_column(str), pattern, limit)) @ignore_unicode_prefix @@ -2719,6 +2733,39 @@ def udf(f=None, returnType=StringType()): | 8| JOHN DOE| 22| +----------+--------------+------------+ """ + + # The following table shows most of Python data and SQL type conversions in normal UDFs that + # are not yet visible to the user. Some of behaviors are buggy and might be changed in the near + # future. The table might have to be eventually documented externally. + # Please see SPARK-25666's PR to see the codes in order to generate the table below. + # + # +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa + # |SQL Type \ Python Value(Type)|None(NoneType)|True(bool)|1(int)|1(long)| a(str)| a(unicode)| 1970-01-01(date)|1970-01-01 00:00:00(datetime)|1.0(float)|array('i', [1])(array)|[1](list)| (1,)(tuple)| ABC(bytearray)| 1(Decimal)|{'a': 1}(dict)|Row(kwargs=1)(Row)|Row(namedtuple=1)(Row)| # noqa + # +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa + # | boolean| None| True| None| None| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa + # | tinyint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa + # | smallint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa + # | int| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa + # | bigint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa + # | string| None| u'true'| u'1'| u'1'| u'a'| u'a'|u'java.util.Grego...| u'java.util.Grego...| u'1.0'| u'[I@24a83055'| u'[1]'|u'[Ljava.lang.Obj...| u'[B@49093632'| u'1'| u'{a=1}'| X| X| # noqa + # | date| None| X| X| X| X| X|datetime.date(197...| datetime.date(197...| X| X| X| X| X| X| X| X| X| # noqa + # | timestamp| None| X| X| X| X| X| X| datetime.datetime...| X| X| X| X| X| X| X| X| X| # noqa + # | float| None| None| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa + # | double| None| None| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa + # | array| None| None| None| None| None| None| None| None| None| [1]| [1]| [1]| [65, 66, 67]| None| None| X| X| # noqa + # | binary| None| None| None| None|bytearray(b'a')|bytearray(b'a')| None| None| None| None| None| None|bytearray(b'ABC')| None| None| X| X| # noqa + # | decimal(10,0)| None| None| None| None| None| None| None| None| None| None| None| None| None|Decimal('1')| None| X| X| # noqa + # | map| None| None| None| None| None| None| None| None| None| None| None| None| None| None| {u'a': 1}| X| X| # noqa + # | struct<_1:int>| None| X| X| X| X| X| X| X| X| X|Row(_1=1)| Row(_1=1)| X| X| Row(_1=None)| Row(_1=1)| Row(_1=1)| # noqa + # +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa + # + # Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be + # used in `returnType`. + # Note: The values inside of the table are generated by `repr`. + # Note: Python 2 is used to generate this table since it is used to check the backward + # compatibility often in practice. + # Note: 'X' means it throws an exception during the conversion. + # decorator @udf, @udf(), @udf(dataType()) if f is None or isinstance(f, (str, DataType)): # If DataType has been passed as a positional argument @@ -2934,6 +2981,12 @@ def pandas_udf(f=None, returnType=None, functionType=None): can fail on special rows, the workaround is to incorporate the condition into the functions. .. note:: The user-defined functions do not take keyword arguments on the calling side. + + .. note:: The data type of returned `pandas.Series` from the user-defined functions should be + matched with defined returnType (see :meth:`types.to_arrow_type` and + :meth:`types.from_arrow_type`). When there is mismatch between them, Spark might do + conversion on returned data. The conversion is not guaranteed to be correct and results + should be checked for accuracy by users. """ # decorator @pandas_udf(returnType, functionType) is_decorator = f is None or isinstance(f, (str, DataType)) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index d3c29d061fc3..85712df5f2ad 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1149,6 +1149,75 @@ def test_infer_schema(self): result = self.spark.sql("SELECT l[0].a from test2 where d['key'].d = '2'") self.assertEqual(1, result.head()[0]) + def test_infer_schema_specification(self): + from decimal import Decimal + + class A(object): + def __init__(self): + self.a = 1 + + data = [ + True, + 1, + "a", + u"a", + datetime.date(1970, 1, 1), + datetime.datetime(1970, 1, 1, 0, 0), + 1.0, + array.array("d", [1]), + [1], + (1, ), + {"a": 1}, + bytearray(1), + Decimal(1), + Row(a=1), + Row("a")(1), + A(), + ] + + df = self.spark.createDataFrame([data]) + actual = list(map(lambda x: x.dataType.simpleString(), df.schema)) + expected = [ + 'boolean', + 'bigint', + 'string', + 'string', + 'date', + 'timestamp', + 'double', + 'array', + 'array', + 'struct<_1:bigint>', + 'map', + 'binary', + 'decimal(38,18)', + 'struct', + 'struct', + 'struct', + ] + self.assertEqual(actual, expected) + + actual = list(df.first()) + expected = [ + True, + 1, + 'a', + u"a", + datetime.date(1970, 1, 1), + datetime.datetime(1970, 1, 1, 0, 0), + 1.0, + [1.0], + [1], + Row(_1=1), + {"a": 1}, + bytearray(b'\x00'), + Decimal('1.000000000000000000'), + Row(a=1), + Row(a=1), + Row(a=1), + ] + self.assertEqual(actual, expected) + def test_infer_schema_not_enough_names(self): df = self.spark.createDataFrame([["a", "b"]], ["col1"]) self.assertEqual(df.columns, ['col1', '_2']) @@ -3603,6 +3672,31 @@ def test_repr_behaviors(self): self.assertEquals(None, df._repr_html_()) self.assertEquals(expected, df.__repr__()) + # SPARK-25591 + def test_same_accumulator_in_udfs(self): + from pyspark.sql.functions import udf + + data_schema = StructType([StructField("a", IntegerType(), True), + StructField("b", IntegerType(), True)]) + data = self.spark.createDataFrame([[1, 2]], schema=data_schema) + + test_accum = self.sc.accumulator(0) + + def first_udf(x): + test_accum.add(1) + return x + + def second_udf(x): + test_accum.add(100) + return x + + func_udf = udf(first_udf, IntegerType()) + func_udf2 = udf(second_udf, IntegerType()) + data = data.withColumn("out1", func_udf(data["a"])) + data = data.withColumn("out2", func_udf2(data["b"])) + data.collect() + self.assertEqual(test_accum.value, 101) + class HiveSparkSubmitTests(SparkSubmitTests): diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index ce42a857d0c0..946601e779d2 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -45,7 +45,7 @@ class DStream(object): for more details on RDDs). DStreams can either be created from live data (such as, data from TCP - sockets, Kafka, Flume, etc.) using a L{StreamingContext} or it can be + sockets, Kafka, etc.) using a L{StreamingContext} or it can be generated by transforming existing DStreams using operations such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each DStream periodically generates a RDD, either diff --git a/python/pyspark/streaming/flume.py b/python/pyspark/streaming/flume.py deleted file mode 100644 index 5de448114ece..000000000000 --- a/python/pyspark/streaming/flume.py +++ /dev/null @@ -1,156 +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. -# - -import sys -if sys.version >= "3": - from io import BytesIO -else: - from StringIO import StringIO -import warnings - -from py4j.protocol import Py4JJavaError - -from pyspark.storagelevel import StorageLevel -from pyspark.serializers import PairDeserializer, NoOpSerializer, UTF8Deserializer, read_int -from pyspark.streaming import DStream - -__all__ = ['FlumeUtils', 'utf8_decoder'] - - -def utf8_decoder(s): - """ Decode the unicode as UTF-8 """ - if s is None: - return None - return s.decode('utf-8') - - -class FlumeUtils(object): - - @staticmethod - def createStream(ssc, hostname, port, - storageLevel=StorageLevel.MEMORY_AND_DISK_2, - enableDecompression=False, - bodyDecoder=utf8_decoder): - """ - Create an input stream that pulls events from Flume. - - :param ssc: StreamingContext object - :param hostname: Hostname of the slave machine to which the flume data will be sent - :param port: Port of the slave machine to which the flume data will be sent - :param storageLevel: Storage level to use for storing the received objects - :param enableDecompression: Should netty server decompress input stream - :param bodyDecoder: A function used to decode body (default is utf8_decoder) - :return: A DStream object - - .. note:: Deprecated in 2.3.0. Flume support is deprecated as of Spark 2.3.0. - See SPARK-22142. - """ - warnings.warn( - "Deprecated in 2.3.0. Flume support is deprecated as of Spark 2.3.0. " - "See SPARK-22142.", - DeprecationWarning) - jlevel = ssc._sc._getJavaStorageLevel(storageLevel) - helper = FlumeUtils._get_helper(ssc._sc) - jstream = helper.createStream(ssc._jssc, hostname, port, jlevel, enableDecompression) - return FlumeUtils._toPythonDStream(ssc, jstream, bodyDecoder) - - @staticmethod - def createPollingStream(ssc, addresses, - storageLevel=StorageLevel.MEMORY_AND_DISK_2, - maxBatchSize=1000, - parallelism=5, - bodyDecoder=utf8_decoder): - """ - Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - This stream will poll the sink for data and will pull events as they are available. - - :param ssc: StreamingContext object - :param addresses: List of (host, port)s on which the Spark Sink is running. - :param storageLevel: Storage level to use for storing the received objects - :param maxBatchSize: The maximum number of events to be pulled from the Spark sink - in a single RPC call - :param parallelism: Number of concurrent requests this stream should send to the sink. - Note that having a higher number of requests concurrently being pulled - will result in this stream using more threads - :param bodyDecoder: A function used to decode body (default is utf8_decoder) - :return: A DStream object - - .. note:: Deprecated in 2.3.0. Flume support is deprecated as of Spark 2.3.0. - See SPARK-22142. - """ - warnings.warn( - "Deprecated in 2.3.0. Flume support is deprecated as of Spark 2.3.0. " - "See SPARK-22142.", - DeprecationWarning) - jlevel = ssc._sc._getJavaStorageLevel(storageLevel) - hosts = [] - ports = [] - for (host, port) in addresses: - hosts.append(host) - ports.append(port) - helper = FlumeUtils._get_helper(ssc._sc) - jstream = helper.createPollingStream( - ssc._jssc, hosts, ports, jlevel, maxBatchSize, parallelism) - return FlumeUtils._toPythonDStream(ssc, jstream, bodyDecoder) - - @staticmethod - def _toPythonDStream(ssc, jstream, bodyDecoder): - ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) - stream = DStream(jstream, ssc, ser) - - def func(event): - headersBytes = BytesIO(event[0]) if sys.version >= "3" else StringIO(event[0]) - headers = {} - strSer = UTF8Deserializer() - for i in range(0, read_int(headersBytes)): - key = strSer.loads(headersBytes) - value = strSer.loads(headersBytes) - headers[key] = value - body = bodyDecoder(event[1]) - return (headers, body) - return stream.map(func) - - @staticmethod - def _get_helper(sc): - try: - return sc._jvm.org.apache.spark.streaming.flume.FlumeUtilsPythonHelper() - except TypeError as e: - if str(e) == "'JavaPackage' object is not callable": - FlumeUtils._printErrorMsg(sc) - raise - - @staticmethod - def _printErrorMsg(sc): - print(""" -________________________________________________________________________________________________ - - Spark Streaming's Flume libraries not found in class path. Try one of the following. - - 1. Include the Flume library and its dependencies with in the - spark-submit command as - - $ bin/spark-submit --packages org.apache.spark:spark-streaming-flume:%s ... - - 2. Download the JAR of the artifact from Maven Central http://search.maven.org/, - Group Id = org.apache.spark, Artifact Id = spark-streaming-flume-assembly, Version = %s. - Then, include the jar in the spark-submit command as - - $ bin/spark-submit --jars ... - -________________________________________________________________________________________________ - -""" % (sc.version, sc.version)) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 5cef621a28e6..4b995c04c07d 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -48,7 +48,6 @@ from pyspark.storagelevel import StorageLevel from pyspark.streaming.context import StreamingContext from pyspark.streaming.kafka import Broker, KafkaUtils, OffsetRange, TopicAndPartition -from pyspark.streaming.flume import FlumeUtils from pyspark.streaming.kinesis import KinesisUtils, InitialPositionInStream from pyspark.streaming.listener import StreamingListener @@ -1301,148 +1300,6 @@ def getKeyAndDoubleMessage(m): self._validateStreamResult({"aa": 1, "bb": 2, "cc": 3}, stream) -class FlumeStreamTests(PySparkStreamingTestCase): - timeout = 20 # seconds - duration = 1 - - def setUp(self): - super(FlumeStreamTests, self).setUp() - self._utils = self.ssc._jvm.org.apache.spark.streaming.flume.FlumeTestUtils() - - def tearDown(self): - if self._utils is not None: - self._utils.close() - self._utils = None - - super(FlumeStreamTests, self).tearDown() - - def _startContext(self, n, compressed): - # Start the StreamingContext and also collect the result - dstream = FlumeUtils.createStream(self.ssc, "localhost", self._utils.getTestPort(), - enableDecompression=compressed) - result = [] - - def get_output(_, rdd): - for event in rdd.collect(): - if len(result) < n: - result.append(event) - dstream.foreachRDD(get_output) - self.ssc.start() - return result - - def _validateResult(self, input, result): - # Validate both the header and the body - header = {"test": "header"} - self.assertEqual(len(input), len(result)) - for i in range(0, len(input)): - self.assertEqual(header, result[i][0]) - self.assertEqual(input[i], result[i][1]) - - def _writeInput(self, input, compressed): - # Try to write input to the receiver until success or timeout - start_time = time.time() - while True: - try: - self._utils.writeInput(input, compressed) - break - except: - if time.time() - start_time < self.timeout: - time.sleep(0.01) - else: - raise - - def test_flume_stream(self): - input = [str(i) for i in range(1, 101)] - result = self._startContext(len(input), False) - self._writeInput(input, False) - self.wait_for(result, len(input)) - self._validateResult(input, result) - - def test_compressed_flume_stream(self): - input = [str(i) for i in range(1, 101)] - result = self._startContext(len(input), True) - self._writeInput(input, True) - self.wait_for(result, len(input)) - self._validateResult(input, result) - - -class FlumePollingStreamTests(PySparkStreamingTestCase): - timeout = 20 # seconds - duration = 1 - maxAttempts = 5 - - def setUp(self): - self._utils = self.sc._jvm.org.apache.spark.streaming.flume.PollingFlumeTestUtils() - - def tearDown(self): - if self._utils is not None: - self._utils.close() - self._utils = None - - def _writeAndVerify(self, ports): - # Set up the streaming context and input streams - ssc = StreamingContext(self.sc, self.duration) - try: - addresses = [("localhost", port) for port in ports] - dstream = FlumeUtils.createPollingStream( - ssc, - addresses, - maxBatchSize=self._utils.eventsPerBatch(), - parallelism=5) - outputBuffer = [] - - def get_output(_, rdd): - for e in rdd.collect(): - outputBuffer.append(e) - - dstream.foreachRDD(get_output) - ssc.start() - self._utils.sendDataAndEnsureAllDataHasBeenReceived() - - self.wait_for(outputBuffer, self._utils.getTotalEvents()) - outputHeaders = [event[0] for event in outputBuffer] - outputBodies = [event[1] for event in outputBuffer] - self._utils.assertOutput(outputHeaders, outputBodies) - finally: - ssc.stop(False) - - def _testMultipleTimes(self, f): - attempt = 0 - while True: - try: - f() - break - except: - attempt += 1 - if attempt >= self.maxAttempts: - raise - else: - import traceback - traceback.print_exc() - - def _testFlumePolling(self): - try: - port = self._utils.startSingleSink() - self._writeAndVerify([port]) - self._utils.assertChannelsAreEmpty() - finally: - self._utils.close() - - def _testFlumePollingMultipleHosts(self): - try: - port = self._utils.startSingleSink() - self._writeAndVerify([port]) - self._utils.assertChannelsAreEmpty() - finally: - self._utils.close() - - def test_flume_polling(self): - self._testMultipleTimes(self._testFlumePolling) - - def test_flume_polling_multiple_hosts(self): - self._testMultipleTimes(self._testFlumePollingMultipleHosts) - - class KinesisStreamTests(PySparkStreamingTestCase): def test_kinesis_stream_api(self): @@ -1531,23 +1388,6 @@ def search_kafka_assembly_jar(): return jars[0] -def search_flume_assembly_jar(): - SPARK_HOME = os.environ["SPARK_HOME"] - flume_assembly_dir = os.path.join(SPARK_HOME, "external/flume-assembly") - jars = search_jar(flume_assembly_dir, "spark-streaming-flume-assembly") - if not jars: - raise Exception( - ("Failed to find Spark Streaming Flume assembly jar in %s. " % flume_assembly_dir) + - "You need to build Spark with " - "'build/sbt -Pflume assembly/package streaming-flume-assembly/assembly' or " - "'build/mvn -DskipTests -Pflume package' before running this test.") - elif len(jars) > 1: - raise Exception(("Found multiple Spark Streaming Flume assembly JARs: %s; please " - "remove all but one") % (", ".join(jars))) - else: - return jars[0] - - def _kinesis_asl_assembly_dir(): SPARK_HOME = os.environ["SPARK_HOME"] return os.path.join(SPARK_HOME, "external/kinesis-asl-assembly") @@ -1564,9 +1404,6 @@ def search_kinesis_asl_assembly_jar(): return jars[0] -# Must be same as the variable and condition defined in modules.py -flume_test_environ_var = "ENABLE_FLUME_TESTS" -are_flume_tests_enabled = os.environ.get(flume_test_environ_var) == '1' # Must be same as the variable and condition defined in modules.py kafka_test_environ_var = "ENABLE_KAFKA_0_8_TESTS" are_kafka_tests_enabled = os.environ.get(kafka_test_environ_var) == '1' @@ -1577,15 +1414,14 @@ def search_kinesis_asl_assembly_jar(): if __name__ == "__main__": from pyspark.streaming.tests import * kafka_assembly_jar = search_kafka_assembly_jar() - flume_assembly_jar = search_flume_assembly_jar() kinesis_asl_assembly_jar = search_kinesis_asl_assembly_jar() if kinesis_asl_assembly_jar is None: kinesis_jar_present = False - jars = "%s,%s" % (kafka_assembly_jar, flume_assembly_jar) + jars = kafka_assembly_jar else: kinesis_jar_present = True - jars = "%s,%s,%s" % (kafka_assembly_jar, flume_assembly_jar, kinesis_asl_assembly_jar) + jars = "%s,%s" % (kafka_assembly_jar, kinesis_asl_assembly_jar) existing_args = os.environ.get("PYSPARK_SUBMIT_ARGS", "pyspark-shell") jars_args = "--jars %s" % jars @@ -1593,14 +1429,6 @@ def search_kinesis_asl_assembly_jar(): testcases = [BasicOperationTests, WindowFunctionTests, StreamingContextTests, CheckpointTests, StreamingListenerTests] - if are_flume_tests_enabled: - testcases.append(FlumeStreamTests) - testcases.append(FlumePollingStreamTests) - else: - sys.stderr.write( - "Skipped test_flume_stream (enable by setting environment variable %s=1" - % flume_test_environ_var) - if are_kafka_tests_enabled: testcases.append(KafkaStreamTests) else: diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index 7ae57bf6e42d..1c4dcd547687 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -19,6 +19,7 @@ FROM openjdk:8-alpine ARG spark_jars=jars ARG img_path=kubernetes/dockerfiles +ARG k8s_tests=kubernetes/tests # Before building the docker image, first build and make a Spark distribution following # the instructions in http://spark.apache.org/docs/latest/building-spark.html. @@ -43,7 +44,7 @@ COPY bin /opt/spark/bin COPY sbin /opt/spark/sbin COPY ${img_path}/spark/entrypoint.sh /opt/ COPY examples /opt/spark/examples -COPY kubernetes/tests /opt/spark/tests +COPY ${k8s_tests} /opt/spark/tests COPY data /opt/spark/data ENV SPARK_HOME /opt/spark diff --git a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh index b28b8b82ca01..e26c0b3a39c9 100755 --- a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh +++ b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh @@ -103,4 +103,4 @@ then properties=( ${properties[@]} -Dtest.exclude.tags=$EXCLUDE_TAGS ) fi -$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pkubernetes -Phadoop-2.7 ${properties[@]} +$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pkubernetes ${properties[@]} 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 4a85898ef880..01bdebc000b9 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 @@ -273,19 +273,10 @@ private[spark] class Client( sparkConf.get(ROLLED_LOG_INCLUDE_PATTERN).foreach { includePattern => try { val logAggregationContext = Records.newRecord(classOf[LogAggregationContext]) - - // These two methods were added in Hadoop 2.6.4, so we still need to use reflection to - // avoid compile error when building against Hadoop 2.6.0 ~ 2.6.3. - val setRolledLogsIncludePatternMethod = - logAggregationContext.getClass.getMethod("setRolledLogsIncludePattern", classOf[String]) - setRolledLogsIncludePatternMethod.invoke(logAggregationContext, includePattern) - + logAggregationContext.setRolledLogsIncludePattern(includePattern) sparkConf.get(ROLLED_LOG_EXCLUDE_PATTERN).foreach { excludePattern => - val setRolledLogsExcludePatternMethod = - logAggregationContext.getClass.getMethod("setRolledLogsExcludePattern", classOf[String]) - setRolledLogsExcludePatternMethod.invoke(logAggregationContext, excludePattern) + logAggregationContext.setRolledLogsExcludePattern(excludePattern) } - appContext.setLogAggregationContext(logAggregationContext) } catch { case NonFatal(e) => diff --git a/sql/catalyst/benchmarks/HashBenchmark-results.txt b/sql/catalyst/benchmarks/HashBenchmark-results.txt new file mode 100644 index 000000000000..2459b35c75bb --- /dev/null +++ b/sql/catalyst/benchmarks/HashBenchmark-results.txt @@ -0,0 +1,70 @@ +================================================================================================ +single ints +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For single ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +interpreted version 5615 / 5616 95.6 10.5 1.0X +codegen version 8400 / 8407 63.9 15.6 0.7X +codegen version 64-bit 8139 / 8145 66.0 15.2 0.7X +codegen HiveHash version 7213 / 7348 74.4 13.4 0.8X + + +================================================================================================ +single longs +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For single longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +interpreted version 6053 / 6054 88.7 11.3 1.0X +codegen version 9367 / 9369 57.3 17.4 0.6X +codegen version 64-bit 8041 / 8051 66.8 15.0 0.8X +codegen HiveHash version 7546 / 7575 71.1 14.1 0.8X + + +================================================================================================ +normal +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For normal: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +interpreted version 3181 / 3182 0.7 1517.0 1.0X +codegen version 2403 / 2403 0.9 1145.7 1.3X +codegen version 64-bit 915 / 916 2.3 436.2 3.5X +codegen HiveHash version 4505 / 4527 0.5 2148.3 0.7X + + +================================================================================================ +array +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +interpreted version 1828 / 1844 0.1 13946.1 1.0X +codegen version 3678 / 3804 0.0 28058.2 0.5X +codegen version 64-bit 2925 / 2931 0.0 22317.8 0.6X +codegen HiveHash version 1216 / 1217 0.1 9280.0 1.5X + + +================================================================================================ +map +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash For map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +interpreted version 0 / 0 44.3 22.6 1.0X +codegen version 176 / 176 0.0 42978.8 0.0X +codegen version 64-bit 173 / 175 0.0 42214.3 0.0X +codegen HiveHash version 44 / 44 0.1 10659.9 0.0X + + diff --git a/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt b/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt new file mode 100644 index 000000000000..a4304ee3b5f6 --- /dev/null +++ b/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt @@ -0,0 +1,77 @@ +================================================================================================ +Benchmark for MurMurHash 3 and xxHash64 +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 8: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Murmur3_x86_32 16 / 16 127.7 7.8 1.0X +xxHash 64-bit 23 / 23 90.7 11.0 0.7X +HiveHasher 16 / 16 134.8 7.4 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 16: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Murmur3_x86_32 26 / 26 79.5 12.6 1.0X +xxHash 64-bit 26 / 27 79.3 12.6 1.0X +HiveHasher 30 / 30 70.1 14.3 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 24: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Murmur3_x86_32 36 / 36 58.1 17.2 1.0X +xxHash 64-bit 30 / 30 70.2 14.2 1.2X +HiveHasher 45 / 45 46.4 21.5 0.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 31: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Murmur3_x86_32 50 / 50 41.8 23.9 1.0X +xxHash 64-bit 43 / 43 49.3 20.3 1.2X +HiveHasher 58 / 58 35.9 27.8 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 95: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Murmur3_x86_32 132 / 132 15.9 62.7 1.0X +xxHash 64-bit 79 / 79 26.7 37.5 1.7X +HiveHasher 198 / 199 10.6 94.6 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 287: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Murmur3_x86_32 334 / 334 6.3 159.3 1.0X +xxHash 64-bit 126 / 126 16.7 59.9 2.7X +HiveHasher 633 / 634 3.3 302.0 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 1055: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Murmur3_x86_32 1149 / 1149 1.8 547.9 1.0X +xxHash 64-bit 327 / 327 6.4 155.9 3.5X +HiveHasher 2338 / 2346 0.9 1114.6 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 2079: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Murmur3_x86_32 2215 / 2216 0.9 1056.1 1.0X +xxHash 64-bit 554 / 554 3.8 264.0 4.0X +HiveHasher 4609 / 4609 0.5 2197.5 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash byte arrays with length 8223: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Murmur3_x86_32 8633 / 8643 0.2 4116.3 1.0X +xxHash 64-bit 1891 / 1892 1.1 901.6 4.6X +HiveHasher 18206 / 18206 0.1 8681.3 0.5X + + diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 16665eb0d737..056998630b09 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -468,7 +468,7 @@ joinType joinCriteria : ON booleanExpression - | USING '(' identifier (',' identifier)* ')' + | USING identifierList ; sample diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java index 551443a11298..460513816dfd 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.catalyst.expressions; +import java.io.Closeable; import java.io.IOException; import org.apache.spark.memory.MemoryConsumer; @@ -45,7 +46,7 @@ * page requires an average size for key value pairs to be larger than 1024 bytes. * */ -public abstract class RowBasedKeyValueBatch extends MemoryConsumer { +public abstract class RowBasedKeyValueBatch extends MemoryConsumer implements Closeable { protected final Logger logger = LoggerFactory.getLogger(RowBasedKeyValueBatch.class); private static final int DEFAULT_CAPACITY = 1 << 16; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d72e512e0df5..7f641ace4629 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2150,8 +2150,10 @@ class Analyzer( // TODO: skip null handling for not-nullable primitive inputs after we can completely // trust the `nullable` information. + val needsNullCheck = (nullable: Boolean, expr: Expression) => + nullable && !expr.isInstanceOf[KnownNotNull] val inputsNullCheck = nullableTypes.zip(inputs) - .filter { case (nullable, _) => !nullable } + .filter { case (nullableType, expr) => needsNullCheck(!nullableType, expr) } .map { case (_, expr) => IsNull(expr) } .reduceLeftOption[Expression]((e1, e2) => Or(e1, e2)) // Once we add an `If` check above the udf, it is safe to mark those checked inputs diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index bf0c35fe6101..4f5ea1e95f83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -157,7 +157,7 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi arguments = """ Arguments: * str - a string expression - * regexp - a string expression. The pattern string should be a Java regular expression. + * regexp - a string expression. The regex string should be a Java regular expression. Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL parser. For example, to match "\abc", a regular expression for `regexp` can be @@ -229,33 +229,53 @@ case class RLike(left: Expression, right: Expression) extends StringRegexExpress /** - * Splits str around pat (pattern is a regular expression). + * Splits str around matches of the given regex. */ @ExpressionDescription( - usage = "_FUNC_(str, regex) - Splits `str` around occurrences that match `regex`.", + usage = "_FUNC_(str, regex, limit) - Splits `str` around occurrences that match `regex`" + + " and returns an array with a length of at most `limit`", + arguments = """ + Arguments: + * str - a string expression to split. + * regex - a string representing a regular expression. The regex string should be a + Java regular expression. + * limit - an integer expression which controls the number of times the regex is applied. + * limit > 0: The resulting array's length will not be more than `limit`, + and the resulting array's last entry will contain all input + beyond the last matched regex. + * limit <= 0: `regex` will be applied as many times as possible, and + the resulting array can be of any size. + """, examples = """ Examples: > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]'); ["one","two","three",""] + > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]', -1); + ["one","two","three",""] + > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]', 2); + ["one","twoBthreeC"] """) -case class StringSplit(str: Expression, pattern: Expression) - extends BinaryExpression with ImplicitCastInputTypes { +case class StringSplit(str: Expression, regex: Expression, limit: Expression) + extends TernaryExpression with ImplicitCastInputTypes { - override def left: Expression = str - override def right: Expression = pattern override def dataType: DataType = ArrayType(StringType) - override def inputTypes: Seq[DataType] = Seq(StringType, StringType) + override def inputTypes: Seq[DataType] = Seq(StringType, StringType, IntegerType) + override def children: Seq[Expression] = str :: regex :: limit :: Nil - override def nullSafeEval(string: Any, regex: Any): Any = { - val strings = string.asInstanceOf[UTF8String].split(regex.asInstanceOf[UTF8String], -1) + def this(exp: Expression, regex: Expression) = this(exp, regex, Literal(-1)); + + override def nullSafeEval(string: Any, regex: Any, limit: Any): Any = { + val strings = string.asInstanceOf[UTF8String].split( + regex.asInstanceOf[UTF8String], limit.asInstanceOf[Int]) new GenericArrayData(strings.asInstanceOf[Array[Any]]) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val arrayClass = classOf[GenericArrayData].getName - nullSafeCodeGen(ctx, ev, (str, pattern) => + nullSafeCodeGen(ctx, ev, (str, regex, limit) => { // Array in java is covariant, so we don't need to cast UTF8String[] to Object[]. - s"""${ev.value} = new $arrayClass($str.split($pattern, -1));""") + s"""${ev.value} = new $arrayClass($str.split($regex,$limit));""".stripMargin + }) } override def prettyName: String = "split" 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 da12a6519bd2..672bffcfc0ca 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 @@ -394,6 +394,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Filter(expression(ctx), plan) } + def withHaving(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = { + // Note that we add a cast to non-predicate expressions. If the expression itself is + // already boolean, the optimizer will get rid of the unnecessary cast. + val predicate = expression(ctx) match { + case p: Predicate => p + case e => Cast(e, BooleanType) + } + Filter(predicate, plan) + } + + // Expressions. val expressions = Option(namedExpressionSeq).toSeq .flatMap(_.namedExpression.asScala) @@ -446,30 +457,34 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case e: NamedExpression => e case e: Expression => UnresolvedAlias(e) } - val withProject = if (aggregation != null) { - withAggregation(aggregation, namedExpressions, withFilter) - } else if (namedExpressions.nonEmpty) { + + def createProject() = if (namedExpressions.nonEmpty) { Project(namedExpressions, withFilter) } else { withFilter } - // Having - val withHaving = withProject.optional(having) { - // Note that we add a cast to non-predicate expressions. If the expression itself is - // already boolean, the optimizer will get rid of the unnecessary cast. - val predicate = expression(having) match { - case p: Predicate => p - case e => Cast(e, BooleanType) + val withProject = if (aggregation == null && having != null) { + if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) { + // If the legacy conf is set, treat HAVING without GROUP BY as WHERE. + withHaving(having, createProject()) + } else { + // According to SQL standard, HAVING without GROUP BY means global aggregate. + withHaving(having, Aggregate(Nil, namedExpressions, withFilter)) } - Filter(predicate, withProject) + } else if (aggregation != null) { + val aggregate = withAggregation(aggregation, namedExpressions, withFilter) + aggregate.optionalMap(having)(withHaving) + } else { + // When hitting this branch, `having` must be null. + createProject() } // Distinct val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) { - Distinct(withHaving) + Distinct(withProject) } else { - withHaving + withProject } // Window @@ -701,7 +716,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Resolve the join type and join condition val (joinType, condition) = Option(join.joinCriteria) match { case Some(c) if c.USING != null => - (UsingJoin(baseJoinType, c.identifier.asScala.map(_.getText)), None) + (UsingJoin(baseJoinType, visitIdentifierList(c.identifierList)), None) case Some(c) if c.booleanExpression != null => (baseJoinType, Option(expression(c.booleanExpression))) case None if join.NATURAL != null => 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 b699707d8523..da70d7da7351 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 @@ -1567,6 +1567,14 @@ object SQLConf { .internal() .booleanConf .createWithDefault(false) + + val LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE = + buildConf("spark.sql.legacy.parser.havingWithoutGroupByAsWhere") + .internal() + .doc("If it is set to true, the parser will treat HAVING without GROUP BY as a normal " + + "WHERE, which does not follow SQL standard.") + .booleanConf + .createWithDefault(false) } /** 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 2da87113c622..8da778800bb9 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 @@ -123,9 +123,8 @@ public void tearDown() { @Test public void emptyBatch() throws Exception { - RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - try { + try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { Assert.assertEquals(0, batch.numRows()); try { batch.getKeyRow(-1); @@ -152,31 +151,24 @@ public void emptyBatch() throws Exception { // Expected exception; do nothing. } Assert.assertFalse(batch.rowIterator().next()); - } finally { - batch.close(); } } @Test - public void batchType() throws Exception { - RowBasedKeyValueBatch batch1 = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - RowBasedKeyValueBatch batch2 = RowBasedKeyValueBatch.allocate(fixedKeySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - try { + public void batchType() { + try (RowBasedKeyValueBatch batch1 = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + RowBasedKeyValueBatch batch2 = RowBasedKeyValueBatch.allocate(fixedKeySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { Assert.assertEquals(batch1.getClass(), VariableLengthRowBasedKeyValueBatch.class); Assert.assertEquals(batch2.getClass(), FixedLengthRowBasedKeyValueBatch.class); - } finally { - batch1.close(); - batch2.close(); } } @Test public void setAndRetrieve() { - RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - try { + try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { UnsafeRow ret1 = appendRow(batch, makeKeyRow(1, "A"), makeValueRow(1, 1)); Assert.assertTrue(checkValue(ret1, 1, 1)); UnsafeRow ret2 = appendRow(batch, makeKeyRow(2, "B"), makeValueRow(2, 2)); @@ -204,33 +196,27 @@ public void setAndRetrieve() { } catch (AssertionError e) { // Expected exception; do nothing. } - } finally { - batch.close(); } } @Test public void setUpdateAndRetrieve() { - RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - try { + try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { appendRow(batch, makeKeyRow(1, "A"), makeValueRow(1, 1)); Assert.assertEquals(1, batch.numRows()); UnsafeRow retrievedValue = batch.getValueRow(0); updateValueRow(retrievedValue, 2, 2); UnsafeRow retrievedValue2 = batch.getValueRow(0); Assert.assertTrue(checkValue(retrievedValue2, 2, 2)); - } finally { - batch.close(); } } @Test public void iteratorTest() throws Exception { - RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - try { + try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { appendRow(batch, makeKeyRow(1, "A"), makeValueRow(1, 1)); appendRow(batch, makeKeyRow(2, "B"), makeValueRow(2, 2)); appendRow(batch, makeKeyRow(3, "C"), makeValueRow(3, 3)); @@ -253,16 +239,13 @@ public void iteratorTest() throws Exception { Assert.assertTrue(checkKey(key3, 3, "C")); Assert.assertTrue(checkValue(value3, 3, 3)); Assert.assertFalse(iterator.next()); - } finally { - batch.close(); } } @Test public void fixedLengthTest() throws Exception { - RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(fixedKeySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - try { + try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(fixedKeySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { appendRow(batch, makeKeyRow(11, 11), makeValueRow(1, 1)); appendRow(batch, makeKeyRow(22, 22), makeValueRow(2, 2)); appendRow(batch, makeKeyRow(33, 33), makeValueRow(3, 3)); @@ -293,16 +276,13 @@ public void fixedLengthTest() throws Exception { Assert.assertTrue(checkKey(key3, 33, 33)); Assert.assertTrue(checkValue(value3, 3, 3)); Assert.assertFalse(iterator.next()); - } finally { - batch.close(); } } @Test public void appendRowUntilExceedingCapacity() throws Exception { - RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, 10); - try { + try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, 10)) { UnsafeRow key = makeKeyRow(1, "A"); UnsafeRow value = makeValueRow(1, 1); for (int i = 0; i < 10; i++) { @@ -321,8 +301,6 @@ public void appendRowUntilExceedingCapacity() throws Exception { Assert.assertTrue(checkValue(value1, 1, 1)); } Assert.assertFalse(iterator.next()); - } finally { - batch.close(); } } @@ -330,9 +308,8 @@ public void appendRowUntilExceedingCapacity() throws Exception { public void appendRowUntilExceedingPageSize() throws Exception { // Use default size or spark.buffer.pageSize if specified int pageSizeToUse = (int) memoryManager.pageSizeBytes(); - RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, pageSizeToUse); //enough capacity - try { + try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, pageSizeToUse)) { UnsafeRow key = makeKeyRow(1, "A"); UnsafeRow value = makeValueRow(1, 1); int recordLength = 8 + key.getSizeInBytes() + value.getSizeInBytes() + 8; @@ -356,49 +333,44 @@ public void appendRowUntilExceedingPageSize() throws Exception { Assert.assertTrue(checkValue(value1, 1, 1)); } Assert.assertFalse(iterator.next()); - } finally { - batch.close(); } } @Test public void failureToAllocateFirstPage() throws Exception { memoryManager.limit(1024); - RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - try { + try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { UnsafeRow key = makeKeyRow(1, "A"); UnsafeRow value = makeValueRow(11, 11); UnsafeRow ret = appendRow(batch, key, value); Assert.assertNull(ret); Assert.assertFalse(batch.rowIterator().next()); - } finally { - batch.close(); } } @Test public void randomizedTest() { - RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - int numEntry = 100; - long[] expectedK1 = new long[numEntry]; - String[] expectedK2 = new String[numEntry]; - long[] expectedV1 = new long[numEntry]; - long[] expectedV2 = new long[numEntry]; - - for (int i = 0; i < numEntry; i++) { - long k1 = rand.nextLong(); - String k2 = getRandomString(rand.nextInt(256)); - long v1 = rand.nextLong(); - long v2 = rand.nextLong(); - appendRow(batch, makeKeyRow(k1, k2), makeValueRow(v1, v2)); - expectedK1[i] = k1; - expectedK2[i] = k2; - expectedV1[i] = v1; - expectedV2[i] = v2; - } - try { + try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { + int numEntry = 100; + long[] expectedK1 = new long[numEntry]; + String[] expectedK2 = new String[numEntry]; + long[] expectedV1 = new long[numEntry]; + long[] expectedV2 = new long[numEntry]; + + for (int i = 0; i < numEntry; i++) { + long k1 = rand.nextLong(); + String k2 = getRandomString(rand.nextInt(256)); + long v1 = rand.nextLong(); + long v2 = rand.nextLong(); + appendRow(batch, makeKeyRow(k1, k2), makeValueRow(v1, v2)); + expectedK1[i] = k1; + expectedK2[i] = k2; + expectedV1[i] = v1; + expectedV2[i] = v2; + } + for (int j = 0; j < 10000; j++) { int rowId = rand.nextInt(numEntry); if (rand.nextBoolean()) { @@ -410,8 +382,6 @@ public void randomizedTest() { Assert.assertTrue(checkValue(value, expectedV1[rowId], expectedV2[rowId])); } } - } finally { - batch.close(); } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala index 7a2a66c9b1d3..4226ab3773fe 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.benchmark.Benchmark +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection @@ -26,94 +26,87 @@ import org.apache.spark.sql.types._ /** * Benchmark for the previous interpreted hash function(InternalRow.hashCode) vs codegened * hash expressions (Murmur3Hash/xxHash64). + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "catalyst/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "catalyst/test:runMain " + * Results will be written to "benchmarks/HashBenchmark-results.txt". + * }}} */ -object HashBenchmark { +object HashBenchmark extends BenchmarkBase { def test(name: String, schema: StructType, numRows: Int, iters: Int): Unit = { - val generator = RandomDataGenerator.forType(schema, nullable = false).get - val encoder = RowEncoder(schema) - val attrs = schema.toAttributes - val safeProjection = GenerateSafeProjection.generate(attrs, attrs) + runBenchmark(name) { + val generator = RandomDataGenerator.forType(schema, nullable = false).get + val encoder = RowEncoder(schema) + val attrs = schema.toAttributes + val safeProjection = GenerateSafeProjection.generate(attrs, attrs) - val rows = (1 to numRows).map(_ => - // The output of encoder is UnsafeRow, use safeProjection to turn in into safe format. - safeProjection(encoder.toRow(generator().asInstanceOf[Row])).copy() - ).toArray + val rows = (1 to numRows).map(_ => + // The output of encoder is UnsafeRow, use safeProjection to turn in into safe format. + safeProjection(encoder.toRow(generator().asInstanceOf[Row])).copy() + ).toArray - val benchmark = new Benchmark("Hash For " + name, iters * numRows.toLong) - benchmark.addCase("interpreted version") { _: Int => - var sum = 0 - for (_ <- 0L until iters) { - var i = 0 - while (i < numRows) { - sum += rows(i).hashCode() - i += 1 + val benchmark = new Benchmark("Hash For " + name, iters * numRows.toLong, output = output) + benchmark.addCase("interpreted version") { _: Int => + var sum = 0 + for (_ <- 0L until iters) { + var i = 0 + while (i < numRows) { + sum += rows(i).hashCode() + i += 1 + } } } - } - val getHashCode = UnsafeProjection.create(new Murmur3Hash(attrs) :: Nil, attrs) - benchmark.addCase("codegen version") { _: Int => - var sum = 0 - for (_ <- 0L until iters) { - var i = 0 - while (i < numRows) { - sum += getHashCode(rows(i)).getInt(0) - i += 1 + val getHashCode = UnsafeProjection.create(new Murmur3Hash(attrs) :: Nil, attrs) + benchmark.addCase("codegen version") { _: Int => + var sum = 0 + for (_ <- 0L until iters) { + var i = 0 + while (i < numRows) { + sum += getHashCode(rows(i)).getInt(0) + i += 1 + } } } - } - val getHashCode64b = UnsafeProjection.create(new XxHash64(attrs) :: Nil, attrs) - benchmark.addCase("codegen version 64-bit") { _: Int => - var sum = 0 - for (_ <- 0L until iters) { - var i = 0 - while (i < numRows) { - sum += getHashCode64b(rows(i)).getInt(0) - i += 1 + val getHashCode64b = UnsafeProjection.create(new XxHash64(attrs) :: Nil, attrs) + benchmark.addCase("codegen version 64-bit") { _: Int => + var sum = 0 + for (_ <- 0L until iters) { + var i = 0 + while (i < numRows) { + sum += getHashCode64b(rows(i)).getInt(0) + i += 1 + } } } - } - val getHiveHashCode = UnsafeProjection.create(new HiveHash(attrs) :: Nil, attrs) - benchmark.addCase("codegen HiveHash version") { _: Int => - var sum = 0 - for (_ <- 0L until iters) { - var i = 0 - while (i < numRows) { - sum += getHiveHashCode(rows(i)).getInt(0) - i += 1 + val getHiveHashCode = UnsafeProjection.create(new HiveHash(attrs) :: Nil, attrs) + benchmark.addCase("codegen HiveHash version") { _: Int => + var sum = 0 + for (_ <- 0L until iters) { + var i = 0 + while (i < numRows) { + sum += getHiveHashCode(rows(i)).getInt(0) + i += 1 + } } } - } - benchmark.run() + benchmark.run() + } } - def main(args: Array[String]): Unit = { + override def runBenchmarkSuite(): Unit = { val singleInt = new StructType().add("i", IntegerType) - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash For single ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - interpreted version 3262 / 3267 164.6 6.1 1.0X - codegen version 6448 / 6718 83.3 12.0 0.5X - codegen version 64-bit 6088 / 6154 88.2 11.3 0.5X - codegen HiveHash version 4732 / 4745 113.5 8.8 0.7X - */ test("single ints", singleInt, 1 << 15, 1 << 14) val singleLong = new StructType().add("i", LongType) - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash For single longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - interpreted version 3716 / 3726 144.5 6.9 1.0X - codegen version 7706 / 7732 69.7 14.4 0.5X - codegen version 64-bit 6370 / 6399 84.3 11.9 0.6X - codegen HiveHash version 4924 / 5026 109.0 9.2 0.8X - */ test("single longs", singleLong, 1 << 15, 1 << 14) val normal = new StructType() @@ -131,45 +124,18 @@ object HashBenchmark { .add("binary", BinaryType) .add("date", DateType) .add("timestamp", TimestampType) - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash For normal: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - interpreted version 2985 / 3013 0.7 1423.4 1.0X - codegen version 2422 / 2434 0.9 1155.1 1.2X - codegen version 64-bit 856 / 920 2.5 408.0 3.5X - codegen HiveHash version 4501 / 4979 0.5 2146.4 0.7X - */ test("normal", normal, 1 << 10, 1 << 11) val arrayOfInt = ArrayType(IntegerType) val array = new StructType() .add("array", arrayOfInt) .add("arrayOfArray", ArrayType(arrayOfInt)) - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash For array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - interpreted version 3100 / 3555 0.0 23651.8 1.0X - codegen version 5779 / 5865 0.0 44088.4 0.5X - codegen version 64-bit 4738 / 4821 0.0 36151.7 0.7X - codegen HiveHash version 2200 / 2246 0.1 16785.9 1.4X - */ test("array", array, 1 << 8, 1 << 9) val mapOfInt = MapType(IntegerType, IntegerType) val map = new StructType() .add("map", mapOfInt) .add("mapOfMap", MapType(IntegerType, mapOfInt)) - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash For map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - interpreted version 0 / 0 48.1 20.8 1.0X - codegen version 257 / 275 0.0 62768.7 0.0X - codegen version 64-bit 226 / 240 0.0 55224.5 0.0X - codegen HiveHash version 89 / 96 0.0 21708.8 0.0X - */ test("map", map, 1 << 6, 1 << 6) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala index a60eb20d9ede..7dc865d85af0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala @@ -19,15 +19,24 @@ package org.apache.spark.sql import java.util.Random -import org.apache.spark.benchmark.Benchmark +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.sql.catalyst.expressions.{HiveHasher, XXH64} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 /** * Synthetic benchmark for MurMurHash 3 and xxHash64. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "catalyst/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "catalyst/test:runMain " + * Results will be written to "benchmarks/HashByteArrayBenchmark-results.txt". + * }}} */ -object HashByteArrayBenchmark { +object HashByteArrayBenchmark extends BenchmarkBase { def test(length: Int, seed: Long, numArrays: Int, iters: Int): Unit = { val random = new Random(seed) val arrays = Array.fill[Array[Byte]](numArrays) { @@ -36,8 +45,8 @@ object HashByteArrayBenchmark { bytes } - val benchmark = - new Benchmark("Hash byte arrays with length " + length, iters * numArrays.toLong) + val benchmark = new Benchmark( + "Hash byte arrays with length " + length, iters * numArrays.toLong, output = output) benchmark.addCase("Murmur3_x86_32") { _: Int => var sum = 0L for (_ <- 0L until iters) { @@ -74,96 +83,17 @@ object HashByteArrayBenchmark { benchmark.run() } - def main(args: Array[String]): Unit = { - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash byte arrays with length 8: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Murmur3_x86_32 12 / 16 174.3 5.7 1.0X - xxHash 64-bit 17 / 22 120.0 8.3 0.7X - HiveHasher 13 / 15 162.1 6.2 0.9X - */ - test(8, 42L, 1 << 10, 1 << 11) - - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash byte arrays with length 16: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Murmur3_x86_32 19 / 22 107.6 9.3 1.0X - xxHash 64-bit 20 / 24 104.6 9.6 1.0X - HiveHasher 24 / 28 87.0 11.5 0.8X - */ - test(16, 42L, 1 << 10, 1 << 11) - - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash byte arrays with length 24: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Murmur3_x86_32 28 / 32 74.8 13.4 1.0X - xxHash 64-bit 24 / 29 87.3 11.5 1.2X - HiveHasher 36 / 41 57.7 17.3 0.8X - */ - test(24, 42L, 1 << 10, 1 << 11) - - // Add 31 to all arrays to create worse case alignment for xxHash. - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash byte arrays with length 31: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Murmur3_x86_32 41 / 45 51.1 19.6 1.0X - xxHash 64-bit 36 / 44 58.8 17.0 1.2X - HiveHasher 49 / 54 42.6 23.5 0.8X - */ - test(31, 42L, 1 << 10, 1 << 11) - - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash byte arrays with length 95: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Murmur3_x86_32 100 / 110 21.0 47.7 1.0X - xxHash 64-bit 74 / 78 28.2 35.5 1.3X - HiveHasher 189 / 196 11.1 90.3 0.5X - */ - test(64 + 31, 42L, 1 << 10, 1 << 11) - - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash byte arrays with length 287: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Murmur3_x86_32 299 / 311 7.0 142.4 1.0X - xxHash 64-bit 113 / 122 18.5 54.1 2.6X - HiveHasher 620 / 624 3.4 295.5 0.5X - */ - test(256 + 31, 42L, 1 << 10, 1 << 11) - - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash byte arrays with length 1055: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Murmur3_x86_32 1068 / 1070 2.0 509.1 1.0X - xxHash 64-bit 306 / 315 6.9 145.9 3.5X - HiveHasher 2316 / 2369 0.9 1104.3 0.5X - */ - test(1024 + 31, 42L, 1 << 10, 1 << 11) - - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash byte arrays with length 2079: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Murmur3_x86_32 2252 / 2274 0.9 1074.1 1.0X - xxHash 64-bit 534 / 580 3.9 254.6 4.2X - HiveHasher 4739 / 4786 0.4 2259.8 0.5X - */ - test(2048 + 31, 42L, 1 << 10, 1 << 11) - - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Hash byte arrays with length 8223: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Murmur3_x86_32 9249 / 9586 0.2 4410.5 1.0X - xxHash 64-bit 2897 / 3241 0.7 1381.6 3.2X - HiveHasher 19392 / 20211 0.1 9246.6 0.5X - */ - test(8192 + 31, 42L, 1 << 10, 1 << 11) + override def runBenchmarkSuite(): Unit = { + runBenchmark("Benchmark for MurMurHash 3 and xxHash64") { + test(8, 42L, 1 << 10, 1 << 11) + test(16, 42L, 1 << 10, 1 << 11) + test(24, 42L, 1 << 10, 1 << 11) + test(31, 42L, 1 << 10, 1 << 11) + test(64 + 31, 42L, 1 << 10, 1 << 11) + test(256 + 31, 42L, 1 << 10, 1 << 11) + test(1024 + 31, 42L, 1 << 10, 1 << 11) + test(2048 + 31, 42L, 1 << 10, 1 << 11) + test(8192 + 31, 42L, 1 << 10, 1 << 11) + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index f9facbb71a4e..cf76c92b093b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -351,8 +351,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("SPARK-24891 Fix HandleNullInputsForUDF rule") { val a = testRelation.output(0) val func = (x: Int, y: Int) => x + y - val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil) - val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil) + val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil, nullableTypes = false :: false :: Nil) + val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil, nullableTypes = false :: false :: Nil) val plan = Project(Alias(udf2, "")() :: Nil, testRelation) comparePlans(plan.analyze, plan.analyze.analyze) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 90c0bf7d8b3d..94dee7ea048c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -112,7 +112,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("cast string to timestamp") { - for (tz <- Random.shuffle(ALL_TIMEZONES).take(50)) { + ALL_TIMEZONES.par.foreach { tz => def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { checkEvaluation(cast(Literal(str), TimestampType, Option(tz.getID)), expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 5e8113ac8658..7843003a4aac 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -113,7 +113,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actual.length == 1) val expected = UTF8String.fromString("abc") - if (!checkResult(actual.head, expected, expressions.head.dataType)) { + if (!checkResult(actual.head, expected, expressions.head)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -126,7 +126,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actual.length == 1) val expected = UnsafeArrayData.fromPrimitiveArray(Array.fill(length)(true)) - if (!checkResult(actual.head, expected, expressions.head.dataType)) { + if (!checkResult(actual.head, expected, expressions.head)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -142,7 +142,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actual.length == 1) val expected = ArrayBasedMapData((0 until length).toArray, Array.fill(length)(true)) - if (!checkResult(actual.head, expected, expressions.head.dataType)) { + if (!checkResult(actual.head, expected, expressions.head)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -154,7 +154,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) val expected = Seq(InternalRow(Seq.fill(length)(true): _*)) - if (!checkResult(actual, expected, expressions.head.dataType)) { + if (!checkResult(actual, expected, expressions.head)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -170,7 +170,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actual.length == 1) val expected = InternalRow(Seq.fill(length)(true): _*) - if (!checkResult(actual.head, expected, expressions.head.dataType)) { + if (!checkResult(actual.head, expected, expressions.head)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -375,7 +375,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actualOr.length == 1) val expectedOr = false - if (!checkResult(actualOr.head, expectedOr, exprOr.dataType)) { + if (!checkResult(actualOr.head, expectedOr, exprOr)) { fail(s"Incorrect Evaluation: expressions: $exprOr, actual: $actualOr, expected: $expectedOr") } @@ -389,7 +389,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actualAnd.length == 1) val expectedAnd = false - if (!checkResult(actualAnd.head, expectedAnd, exprAnd.dataType)) { + if (!checkResult(actualAnd.head, expectedAnd, exprAnd)) { fail( s"Incorrect Evaluation: expressions: $exprAnd, actual: $actualAnd, expected: $expectedAnd") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index b5986aac6555..da18475276a1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -69,11 +69,22 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa /** * Check the equality between result of expression and expected value, it will handle - * Array[Byte], Spread[Double], MapData and Row. + * Array[Byte], Spread[Double], MapData and Row. Also check whether nullable in expression is + * true if result is null */ - protected def checkResult(result: Any, expected: Any, exprDataType: DataType): Boolean = { + protected def checkResult(result: Any, expected: Any, expression: Expression): Boolean = { + checkResult(result, expected, expression.dataType, expression.nullable) + } + + protected def checkResult( + result: Any, + expected: Any, + exprDataType: DataType, + exprNullable: Boolean): Boolean = { val dataType = UserDefinedType.sqlType(exprDataType) + // The result is null for a non-nullable expression + assert(result != null || exprNullable, "exprNullable should be true if result is null") (result, expected) match { case (result: Array[Byte], expected: Array[Byte]) => java.util.Arrays.equals(result, expected) @@ -83,24 +94,24 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa val st = dataType.asInstanceOf[StructType] assert(result.numFields == st.length && expected.numFields == st.length) st.zipWithIndex.forall { case (f, i) => - checkResult(result.get(i, f.dataType), expected.get(i, f.dataType), f.dataType) + checkResult( + result.get(i, f.dataType), expected.get(i, f.dataType), f.dataType, f.nullable) } case (result: ArrayData, expected: ArrayData) => result.numElements == expected.numElements && { - val et = dataType.asInstanceOf[ArrayType].elementType + val ArrayType(et, cn) = dataType.asInstanceOf[ArrayType] var isSame = true var i = 0 while (isSame && i < result.numElements) { - isSame = checkResult(result.get(i, et), expected.get(i, et), et) + isSame = checkResult(result.get(i, et), expected.get(i, et), et, cn) i += 1 } isSame } case (result: MapData, expected: MapData) => - val kt = dataType.asInstanceOf[MapType].keyType - val vt = dataType.asInstanceOf[MapType].valueType - checkResult(result.keyArray, expected.keyArray, ArrayType(kt)) && - checkResult(result.valueArray, expected.valueArray, ArrayType(vt)) + val MapType(kt, vt, vcn) = dataType.asInstanceOf[MapType] + checkResult(result.keyArray, expected.keyArray, ArrayType(kt, false), false) && + checkResult(result.valueArray, expected.valueArray, ArrayType(vt, vcn), false) case (result: Double, expected: Double) => if (expected.isNaN) result.isNaN else expected == result case (result: Float, expected: Float) => @@ -175,7 +186,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa val actual = try evaluateWithoutCodegen(expression, inputRow) catch { case e: Exception => fail(s"Exception evaluating $expression", e) } - if (!checkResult(actual, expected, expression.dataType)) { + if (!checkResult(actual, expected, expression)) { val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" fail(s"Incorrect evaluation (codegen off): $expression, " + s"actual: $actual, " + @@ -191,7 +202,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa for (fallbackMode <- modes) { withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> fallbackMode.toString) { val actual = evaluateWithMutableProjection(expression, inputRow) - if (!checkResult(actual, expected, expression.dataType)) { + if (!checkResult(actual, expected, expression)) { val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" fail(s"Incorrect evaluation (fallback mode = $fallbackMode): $expression, " + s"actual: $actual, expected: $expected$input") @@ -221,6 +232,12 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa val unsafeRow = evaluateWithUnsafeProjection(expression, inputRow) val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" + val dataType = expression.dataType + if (!checkResult(unsafeRow.get(0, dataType), expected, dataType, expression.nullable)) { + fail("Incorrect evaluation in unsafe mode (fallback mode = $fallbackMode): " + + s"$expression, actual: $unsafeRow, expected: $expected, " + + s"dataType: $dataType, nullable: ${expression.nullable}") + } if (expected == null) { if (!unsafeRow.isNullAt(0)) { val expectedRow = InternalRow(expected, expected) @@ -229,8 +246,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa } } else { val lit = InternalRow(expected, expected) - val expectedRow = - UnsafeProjection.create(Array(expression.dataType, expression.dataType)).apply(lit) + val expectedRow = UnsafeProjection.create(Array(dataType, dataType)).apply(lit) if (unsafeRow != expectedRow) { fail(s"Incorrect evaluation in unsafe mode (fallback mode = $fallbackMode): " + s"$expression, actual: $unsafeRow, expected: $expectedRow$input") @@ -280,7 +296,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa expression) plan.initialize(0) var actual = plan(inputRow).get(0, expression.dataType) - assert(checkResult(actual, expected, expression.dataType)) + assert(checkResult(actual, expected, expression)) plan = generateProject( GenerateUnsafeProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), @@ -288,7 +304,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa plan.initialize(0) actual = FromUnsafeProjection(expression.dataType :: Nil)( plan(inputRow)).get(0, expression.dataType) - assert(checkResult(actual, expected, expression.dataType)) + assert(checkResult(actual, expected, expression)) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala index 7c7c4cccee25..54ef9641bee0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodegenFallback, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.types.{DataType, IntegerType} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} +import org.apache.spark.sql.types.{DataType, IntegerType, MapType} /** * A test suite for testing [[ExpressionEvalHelper]]. @@ -35,6 +36,13 @@ class ExpressionEvalHelperSuite extends SparkFunSuite with ExpressionEvalHelper val e = intercept[RuntimeException] { checkEvaluation(BadCodegenExpression(), 10) } assert(e.getMessage.contains("some_variable")) } + + test("SPARK-25388: checkEvaluation should fail if nullable in DataType is incorrect") { + val e = intercept[RuntimeException] { + checkEvaluation(MapIncorrectDataTypeExpression(), Map(3 -> 7, 6 -> null)) + } + assert(e.getMessage.contains("and exprNullable was")) + } } /** @@ -53,3 +61,18 @@ case class BadCodegenExpression() extends LeafExpression { } override def dataType: DataType = IntegerType } + +/** + * An expression that returns a MapData with incorrect DataType whose valueContainsNull is false + * while its value includes null + */ +case class MapIncorrectDataTypeExpression() extends LeafExpression with CodegenFallback { + override def nullable: Boolean = false + override def eval(input: InternalRow): Any = { + val keys = new GenericArrayData(Array(3, 6)) + val values = new GenericArrayData(Array(7, null)) + new ArrayBasedMapData(keys, values) + } + // since values includes null, valueContainsNull must be true + override def dataType: DataType = MapType(IntegerType, IntegerType, valueContainsNull = false) +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index d532dc4f7719..06fb73ad8392 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -225,11 +225,18 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val row3 = create_row("aa2bb3cc", null) checkEvaluation( - StringSplit(Literal("aa2bb3cc"), Literal("[1-9]+")), Seq("aa", "bb", "cc"), row1) + StringSplit(Literal("aa2bb3cc"), Literal("[1-9]+"), -1), Seq("aa", "bb", "cc"), row1) checkEvaluation( - StringSplit(s1, s2), Seq("aa", "bb", "cc"), row1) - checkEvaluation(StringSplit(s1, s2), null, row2) - checkEvaluation(StringSplit(s1, s2), null, row3) + StringSplit(Literal("aa2bb3cc"), Literal("[1-9]+"), 2), Seq("aa", "bb3cc"), row1) + // limit = 0 should behave just like limit = -1 + checkEvaluation( + StringSplit(Literal("aacbbcddc"), Literal("c"), 0), Seq("aa", "bb", "dd", ""), row1) + checkEvaluation( + StringSplit(Literal("aacbbcddc"), Literal("c"), -1), Seq("aa", "bb", "dd", ""), row1) + checkEvaluation( + StringSplit(s1, s2, -1), Seq("aa", "bb", "cc"), row1) + checkEvaluation(StringSplit(s1, s2, -1), null, row2) + checkEvaluation(StringSplit(s1, s2, -1), null, row3) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 422bf97e30e7..f5da90f7cf0c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -108,7 +108,7 @@ class PlanParserSuite extends AnalysisTest { assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b)) assertEqual( "select a, b from db.c having x < 1", - table("db", "c").select('a, 'b).where('x < 1)) + table("db", "c").groupBy()('a, 'b).where('x < 1)) assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b))) assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b)) assertEqual("select from tbl", OneRowRelation().select('from.as("tbl"))) diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt new file mode 100644 index 000000000000..2d3bae442cc5 --- /dev/null +++ b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt @@ -0,0 +1,269 @@ +================================================================================================ +SQL Single Numeric Column Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 21508 / 22112 0.7 1367.5 1.0X +SQL Json 8705 / 8825 1.8 553.4 2.5X +SQL Parquet Vectorized 157 / 186 100.0 10.0 136.7X +SQL Parquet MR 1789 / 1794 8.8 113.8 12.0X +SQL ORC Vectorized 156 / 166 100.9 9.9 138.0X +SQL ORC Vectorized with copy 218 / 225 72.1 13.9 98.6X +SQL ORC MR 1448 / 1492 10.9 92.0 14.9X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 202 / 211 77.7 12.9 1.0X +ParquetReader Vectorized -> Row 118 / 120 133.5 7.5 1.7X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 23282 / 23312 0.7 1480.2 1.0X +SQL Json 9187 / 9189 1.7 584.1 2.5X +SQL Parquet Vectorized 204 / 218 77.0 13.0 114.0X +SQL Parquet MR 1941 / 1953 8.1 123.4 12.0X +SQL ORC Vectorized 217 / 225 72.6 13.8 107.5X +SQL ORC Vectorized with copy 279 / 289 56.3 17.8 83.4X +SQL ORC MR 1541 / 1549 10.2 98.0 15.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 288 / 297 54.6 18.3 1.0X +ParquetReader Vectorized -> Row 255 / 257 61.7 16.2 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 24990 / 25012 0.6 1588.8 1.0X +SQL Json 9837 / 9865 1.6 625.4 2.5X +SQL Parquet Vectorized 170 / 180 92.3 10.8 146.6X +SQL Parquet MR 2319 / 2328 6.8 147.4 10.8X +SQL ORC Vectorized 293 / 301 53.7 18.6 85.3X +SQL ORC Vectorized with copy 297 / 309 52.9 18.9 84.0X +SQL ORC MR 1667 / 1674 9.4 106.0 15.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 257 / 274 61.3 16.3 1.0X +ParquetReader Vectorized -> Row 259 / 264 60.8 16.4 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 32537 / 32554 0.5 2068.7 1.0X +SQL Json 12610 / 12668 1.2 801.7 2.6X +SQL Parquet Vectorized 258 / 276 61.0 16.4 126.2X +SQL Parquet MR 2422 / 2435 6.5 154.0 13.4X +SQL ORC Vectorized 378 / 385 41.6 24.0 86.2X +SQL ORC Vectorized with copy 381 / 389 41.3 24.2 85.4X +SQL ORC MR 1797 / 1819 8.8 114.3 18.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 352 / 368 44.7 22.4 1.0X +ParquetReader Vectorized -> Row 351 / 359 44.8 22.3 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 27179 / 27184 0.6 1728.0 1.0X +SQL Json 12578 / 12585 1.3 799.7 2.2X +SQL Parquet Vectorized 161 / 171 97.5 10.3 168.5X +SQL Parquet MR 2361 / 2395 6.7 150.1 11.5X +SQL ORC Vectorized 473 / 480 33.3 30.0 57.5X +SQL ORC Vectorized with copy 478 / 483 32.9 30.4 56.8X +SQL ORC MR 1858 / 1859 8.5 118.2 14.6X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 251 / 255 62.7 15.9 1.0X +ParquetReader Vectorized -> Row 255 / 259 61.8 16.2 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 34797 / 34830 0.5 2212.3 1.0X +SQL Json 17806 / 17828 0.9 1132.1 2.0X +SQL Parquet Vectorized 260 / 269 60.6 16.5 134.0X +SQL Parquet MR 2512 / 2534 6.3 159.7 13.9X +SQL ORC Vectorized 582 / 593 27.0 37.0 59.8X +SQL ORC Vectorized with copy 576 / 584 27.3 36.6 60.4X +SQL ORC MR 2309 / 2313 6.8 146.8 15.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 350 / 363 44.9 22.3 1.0X +ParquetReader Vectorized -> Row 350 / 366 44.9 22.3 1.0X + + +================================================================================================ +Int and String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 22486 / 22590 0.5 2144.5 1.0X +SQL Json 14124 / 14195 0.7 1347.0 1.6X +SQL Parquet Vectorized 2342 / 2347 4.5 223.4 9.6X +SQL Parquet MR 4660 / 4664 2.2 444.4 4.8X +SQL ORC Vectorized 2378 / 2379 4.4 226.8 9.5X +SQL ORC Vectorized with copy 2548 / 2571 4.1 243.0 8.8X +SQL ORC MR 4206 / 4211 2.5 401.1 5.3X + + +================================================================================================ +Repeated String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 12150 / 12178 0.9 1158.7 1.0X +SQL Json 7012 / 7014 1.5 668.7 1.7X +SQL Parquet Vectorized 792 / 796 13.2 75.5 15.3X +SQL Parquet MR 1961 / 1975 5.3 187.0 6.2X +SQL ORC Vectorized 482 / 485 21.8 46.0 25.2X +SQL ORC Vectorized with copy 710 / 715 14.8 67.7 17.1X +SQL ORC MR 2081 / 2083 5.0 198.5 5.8X + + +================================================================================================ +Partitioned Table Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Data column - CSV 31789 / 31791 0.5 2021.1 1.0X +Data column - Json 12873 / 12918 1.2 818.4 2.5X +Data column - Parquet Vectorized 267 / 280 58.9 17.0 119.1X +Data column - Parquet MR 3387 / 3402 4.6 215.3 9.4X +Data column - ORC Vectorized 391 / 453 40.2 24.9 81.2X +Data column - ORC Vectorized with copy 392 / 398 40.2 24.9 81.2X +Data column - ORC MR 2508 / 2512 6.3 159.4 12.7X +Partition column - CSV 6965 / 6977 2.3 442.8 4.6X +Partition column - Json 5563 / 5576 2.8 353.7 5.7X +Partition column - Parquet Vectorized 65 / 78 241.1 4.1 487.2X +Partition column - Parquet MR 1811 / 1811 8.7 115.1 17.6X +Partition column - ORC Vectorized 66 / 73 239.0 4.2 483.0X +Partition column - ORC Vectorized with copy 65 / 70 241.1 4.1 487.3X +Partition column - ORC MR 1775 / 1778 8.9 112.8 17.9X +Both columns - CSV 30032 / 30113 0.5 1909.4 1.1X +Both columns - Json 13941 / 13959 1.1 886.3 2.3X +Both columns - Parquet Vectorized 312 / 330 50.3 19.9 101.7X +Both columns - Parquet MR 3858 / 3862 4.1 245.3 8.2X +Both columns - ORC Vectorized 431 / 437 36.5 27.4 73.8X +Both column - ORC Vectorized with copy 523 / 529 30.1 33.3 60.7X +Both columns - ORC MR 2712 / 2805 5.8 172.4 11.7X + + +================================================================================================ +String with Nulls Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 13525 / 13823 0.8 1289.9 1.0X +SQL Json 9913 / 9921 1.1 945.3 1.4X +SQL Parquet Vectorized 1517 / 1517 6.9 144.7 8.9X +SQL Parquet MR 3996 / 4008 2.6 381.1 3.4X +ParquetReader Vectorized 1120 / 1128 9.4 106.8 12.1X +SQL ORC Vectorized 1203 / 1224 8.7 114.7 11.2X +SQL ORC Vectorized with copy 1639 / 1646 6.4 156.3 8.3X +SQL ORC MR 3720 / 3780 2.8 354.7 3.6X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 15860 / 15877 0.7 1512.5 1.0X +SQL Json 7676 / 7688 1.4 732.0 2.1X +SQL Parquet Vectorized 1072 / 1084 9.8 102.2 14.8X +SQL Parquet MR 2890 / 2897 3.6 275.6 5.5X +ParquetReader Vectorized 1052 / 1053 10.0 100.4 15.1X +SQL ORC Vectorized 1248 / 1248 8.4 119.0 12.7X +SQL ORC Vectorized with copy 1627 / 1637 6.4 155.2 9.7X +SQL ORC MR 3365 / 3369 3.1 320.9 4.7X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 13401 / 13561 0.8 1278.1 1.0X +SQL Json 5253 / 5303 2.0 500.9 2.6X +SQL Parquet Vectorized 233 / 242 45.0 22.2 57.6X +SQL Parquet MR 1791 / 1796 5.9 170.8 7.5X +ParquetReader Vectorized 236 / 238 44.4 22.5 56.7X +SQL ORC Vectorized 453 / 473 23.2 43.2 29.6X +SQL ORC Vectorized with copy 573 / 577 18.3 54.7 23.4X +SQL ORC MR 1846 / 1850 5.7 176.0 7.3X + + +================================================================================================ +Single Column Scan From Wide Columns +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 3147 / 3148 0.3 3001.1 1.0X +SQL Json 2666 / 2693 0.4 2542.9 1.2X +SQL Parquet Vectorized 54 / 58 19.5 51.3 58.5X +SQL Parquet MR 220 / 353 4.8 209.9 14.3X +SQL ORC Vectorized 63 / 77 16.8 59.7 50.3X +SQL ORC Vectorized with copy 63 / 66 16.7 59.8 50.2X +SQL ORC MR 317 / 321 3.3 302.2 9.9X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 50 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 7902 / 7921 0.1 7536.2 1.0X +SQL Json 9467 / 9491 0.1 9028.6 0.8X +SQL Parquet Vectorized 73 / 79 14.3 69.8 108.0X +SQL Parquet MR 239 / 247 4.4 228.0 33.1X +SQL ORC Vectorized 78 / 84 13.4 74.6 101.0X +SQL ORC Vectorized with copy 78 / 88 13.4 74.4 101.3X +SQL ORC MR 910 / 918 1.2 867.6 8.7X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +SQL CSV 13539 / 13543 0.1 12912.0 1.0X +SQL Json 17420 / 17446 0.1 16613.1 0.8X +SQL Parquet Vectorized 103 / 120 10.2 98.1 131.6X +SQL Parquet MR 250 / 258 4.2 238.9 54.1X +SQL ORC Vectorized 99 / 104 10.6 94.6 136.5X +SQL ORC Vectorized with copy 100 / 106 10.5 95.6 135.1X +SQL ORC MR 1653 / 1659 0.6 1576.3 8.2X + + diff --git a/sql/core/benchmarks/MiscBenchmark-results.txt b/sql/core/benchmarks/MiscBenchmark-results.txt new file mode 100644 index 000000000000..85acd5789365 --- /dev/null +++ b/sql/core/benchmarks/MiscBenchmark-results.txt @@ -0,0 +1,120 @@ +================================================================================================ +filter & aggregate without group +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +range/filter/sum wholestage off 47752 / 48952 43.9 22.8 1.0X +range/filter/sum wholestage on 3123 / 3558 671.5 1.5 15.3X + + +================================================================================================ +range/limit/sum +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +range/limit/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +range/limit/sum wholestage off 229 / 236 2288.9 0.4 1.0X +range/limit/sum wholestage on 257 / 267 2041.0 0.5 0.9X + + +================================================================================================ +sample +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +sample with replacement wholestage off 12908 / 13076 10.2 98.5 1.0X +sample with replacement wholestage on 7334 / 7346 17.9 56.0 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +sample without replacement wholestage off 3082 / 3095 42.5 23.5 1.0X +sample without replacement wholestage on 1125 / 1211 116.5 8.6 2.7X + + +================================================================================================ +collect +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +collect 1 million 291 / 311 3.6 277.3 1.0X +collect 2 millions 552 / 564 1.9 526.6 0.5X +collect 4 millions 1104 / 1108 0.9 1053.0 0.3X + + +================================================================================================ +collect limit +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +collect limit 1 million 311 / 340 3.4 296.2 1.0X +collect limit 2 millions 581 / 614 1.8 554.4 0.5X + + +================================================================================================ +generate explode +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate explode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +generate explode array wholestage off 15211 / 15368 1.1 906.6 1.0X +generate explode array wholestage on 10761 / 10776 1.6 641.4 1.4X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate explode map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +generate explode map wholestage off 22128 / 22578 0.8 1318.9 1.0X +generate explode map wholestage on 16421 / 16520 1.0 978.8 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate posexplode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +generate posexplode array wholestage off 17108 / 18019 1.0 1019.7 1.0X +generate posexplode array wholestage on 11715 / 11804 1.4 698.3 1.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate inline array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +generate inline array wholestage off 16358 / 16418 1.0 975.0 1.0X +generate inline array wholestage on 11152 / 11472 1.5 664.7 1.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate big struct array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +generate big struct array wholestage off 708 / 776 0.1 11803.5 1.0X +generate big struct array wholestage on 535 / 589 0.1 8913.9 1.3X + + +================================================================================================ +generate regular generator +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate stack: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +generate stack wholestage off 29082 / 29393 0.6 1733.4 1.0X +generate stack wholestage on 21066 / 21128 0.8 1255.6 1.4X + + diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/BufferedRowIterator.java b/sql/core/src/main/java/org/apache/spark/sql/execution/BufferedRowIterator.java index 74c9c0599271..3d0511b7ba83 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/BufferedRowIterator.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/BufferedRowIterator.java @@ -73,16 +73,6 @@ public void append(InternalRow row) { currentRows.add(row); } - /** - * Returns whether this iterator should stop fetching next row from [[CodegenSupport#inputRDDs]]. - * - * If it returns true, the caller should exit the loop that [[InputAdapter]] generates. - * This interface is mainly used to limit the number of input rows. - */ - public boolean stopEarly() { - return false; - } - /** * Returns whether `processNext()` should stop processing next row from `input` or not. * 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 fe69f252d43e..3af70b5153c8 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 @@ -506,14 +506,11 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { StructType(schema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) val linesWithoutHeader: RDD[String] = maybeFirstLine.map { firstLine => - val parser = new CsvParser(parsedOptions.asParserSettings) - val columnNames = parser.parseLine(firstLine) - CSVDataSource.checkHeaderColumnNames( + val headerChecker = new CSVHeaderChecker( actualSchema, - columnNames, - csvDataset.getClass.getCanonicalName, - parsedOptions.enforceSchema, - sparkSession.sessionState.conf.caseSensitiveAnalysis) + parsedOptions, + source = s"CSV source: $csvDataset") + headerChecker.checkHeaderColumnNames(firstLine) filteredLines.rdd.mapPartitions(CSVUtils.filterHeaderLine(_, firstLine, parsedOptions)) }.getOrElse(filteredLines.rdd) 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 188fce72efac..55e538f49fed 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 @@ -246,8 +246,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { df.sparkSession.sessionState.conf) val options = sessionOptions ++ extraOptions - val relation = DataSourceV2Relation.create(source, options) if (mode == SaveMode.Append) { + val relation = DataSourceV2Relation.create(source, options) runCommand(df.sparkSession, "save") { AppendData.byName(relation, df.logicalPlan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index af6018472cb0..dfb12f272eb2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1098,16 +1098,29 @@ object SQLContext { data: Iterator[_], beanClass: Class[_], attrs: Seq[AttributeReference]): Iterator[InternalRow] = { - val extractors = - JavaTypeInference.getJavaBeanReadableProperties(beanClass).map(_.getReadMethod) - val methodsToConverts = extractors.zip(attrs).map { case (e, attr) => - (e, CatalystTypeConverters.createToCatalystConverter(attr.dataType)) + def createStructConverter(cls: Class[_], fieldTypes: Seq[DataType]): Any => InternalRow = { + val methodConverters = + JavaTypeInference.getJavaBeanReadableProperties(cls).zip(fieldTypes) + .map { case (property, fieldType) => + val method = property.getReadMethod + method -> createConverter(method.getReturnType, fieldType) + } + value => + if (value == null) { + null + } else { + new GenericInternalRow( + methodConverters.map { case (method, converter) => + converter(method.invoke(value)) + }) + } } - data.map { element => - new GenericInternalRow( - methodsToConverts.map { case (e, convert) => convert(e.invoke(element)) } - ): InternalRow + def createConverter(cls: Class[_], dataType: DataType): Any => Any = dataType match { + case struct: StructType => createStructConverter(cls, struct.map(_.dataType)) + case _ => CatalystTypeConverters.createToCatalystConverter(dataType) } + val dataConverter = createStructConverter(beanClass, attrs.map(_.dataType)) + data.map(dataConverter) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala index 48abad907865..9f6b59336080 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala @@ -136,7 +136,7 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { |if ($batch == null) { | $nextBatchFuncName(); |} - |while ($batch != null) { + |while ($limitNotReachedCond $batch != null) { | int $numRows = $batch.numRows(); | int $localEnd = $numRows - $idx; | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { @@ -166,7 +166,7 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { } val inputRow = if (needsUnsafeRowConversion) null else row s""" - |while ($input.hasNext()) { + |while ($limitNotReachedCond $input.hasNext()) { | InternalRow $row = (InternalRow) $input.next(); | $numOutputRows.add(1); | ${consume(ctx, outputVars, inputRow).trim} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 0dc16ba5ce28..f1470e45f129 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -39,7 +39,7 @@ case class SortExec( global: Boolean, child: SparkPlan, testSpillFrequency: Int = 0) - extends UnaryExecNode with CodegenSupport { + extends UnaryExecNode with BlockingOperatorWithCodegen { override def output: Seq[Attribute] = child.output @@ -124,14 +124,6 @@ case class SortExec( // Name of sorter variable used in codegen. private var sorterVariable: String = _ - // The result rows come from the sort buffer, so this operator doesn't need to copy its result - // even if its child does. - override def needCopyResult: Boolean = false - - // Sort operator always consumes all the input rows before outputting any result, so we don't need - // a stop check before sorting. - override def needStopCheck: Boolean = false - override protected def doProduce(ctx: CodegenContext): String = { val needToSort = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "needToSort", v => s"$v = true;") @@ -172,7 +164,7 @@ case class SortExec( | $needToSort = false; | } | - | while ($sortedIterator.hasNext()) { + | while ($limitNotReachedCond $sortedIterator.hasNext()) { | UnsafeRow $outputRow = (UnsafeRow)$sortedIterator.next(); | ${consume(ctx, null, outputRow)} | if (shouldStop()) return; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 1fc4de9e5601..f5aee627fe90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -345,6 +345,61 @@ trait CodegenSupport extends SparkPlan { * don't require shouldStop() in the loop of producing rows. */ def needStopCheck: Boolean = parent.needStopCheck + + /** + * A sequence of checks which evaluate to true if the downstream Limit operators have not received + * enough records and reached the limit. If current node is a data producing node, it can leverage + * this information to stop producing data and complete the data flow earlier. Common data + * producing nodes are leaf nodes like Range and Scan, and blocking nodes like Sort and Aggregate. + * These checks should be put into the loop condition of the data producing loop. + */ + def limitNotReachedChecks: Seq[String] = parent.limitNotReachedChecks + + /** + * A helper method to generate the data producing loop condition according to the + * limit-not-reached checks. + */ + final def limitNotReachedCond: String = { + // InputAdapter is also a leaf node. + val isLeafNode = children.isEmpty || this.isInstanceOf[InputAdapter] + if (!isLeafNode && !this.isInstanceOf[BlockingOperatorWithCodegen]) { + val errMsg = "Only leaf nodes and blocking nodes need to call 'limitNotReachedCond' " + + "in its data producing loop." + if (Utils.isTesting) { + throw new IllegalStateException(errMsg) + } else { + logWarning(s"[BUG] $errMsg Please open a JIRA ticket to report it.") + } + } + if (parent.limitNotReachedChecks.isEmpty) { + "" + } else { + parent.limitNotReachedChecks.mkString("", " && ", " &&") + } + } +} + +/** + * A special kind of operators which support whole stage codegen. Blocking means these operators + * will consume all the inputs first, before producing output. Typical blocking operators are + * sort and aggregate. + */ +trait BlockingOperatorWithCodegen extends CodegenSupport { + + // Blocking operators usually have some kind of buffer to keep the data before producing them, so + // then don't to copy its result even if its child does. + override def needCopyResult: Boolean = false + + // Blocking operators always consume all the input first, so its upstream operators don't need a + // stop check. + override def needStopCheck: Boolean = false + + // Blocking operators need to consume all the inputs before producing any output. This means, + // Limit operator after this blocking operator will never reach its limit during the execution of + // this blocking operator's upstream operators. Here we override this method to return Nil, so + // that upstream operators will not generate useless conditions (which are always evaluated to + // false) for the Limit operators after this blocking operator. + override def limitNotReachedChecks: Seq[String] = Nil } @@ -381,7 +436,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp forceInline = true) val row = ctx.freshName("row") s""" - | while ($input.hasNext() && !stopEarly()) { + | while ($limitNotReachedCond $input.hasNext()) { | InternalRow $row = (InternalRow) $input.next(); | ${consume(ctx, null, row).trim} | if (shouldStop()) return; @@ -677,6 +732,8 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def needStopCheck: Boolean = true + override def limitNotReachedChecks: Seq[String] = Nil + override protected def otherCopyArgs: Seq[AnyRef] = Seq(codegenStageId.asInstanceOf[Integer]) } 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 98adba50b297..6155ec9d30db 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 @@ -45,7 +45,7 @@ case class HashAggregateExec( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode with CodegenSupport { + extends UnaryExecNode with BlockingOperatorWithCodegen { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) @@ -151,14 +151,6 @@ case class HashAggregateExec( child.asInstanceOf[CodegenSupport].inputRDDs() } - // The result rows come from the aggregate buffer, or a single row(no grouping keys), so this - // operator doesn't need to copy its result even if its child does. - override def needCopyResult: Boolean = false - - // Aggregate operator always consumes all the input rows before outputting any result, so we - // don't need a stop check before aggregating. - override def needStopCheck: Boolean = false - protected override def doProduce(ctx: CodegenContext): String = { if (groupingExpressions.isEmpty) { doProduceWithoutKeys(ctx) @@ -705,13 +697,16 @@ case class HashAggregateExec( def outputFromRegularHashMap: String = { s""" - |while ($iterTerm.next()) { + |while ($limitNotReachedCond $iterTerm.next()) { | UnsafeRow $keyTerm = (UnsafeRow) $iterTerm.getKey(); | UnsafeRow $bufferTerm = (UnsafeRow) $iterTerm.getValue(); | $outputFunc($keyTerm, $bufferTerm); - | | if (shouldStop()) return; |} + |$iterTerm.close(); + |if ($sorterTerm == null) { + | $hashMapTerm.free(); + |} """.stripMargin } @@ -728,11 +723,6 @@ case class HashAggregateExec( // output the result $outputFromFastHashMap $outputFromRegularHashMap - - $iterTerm.close(); - if ($sorterTerm == null) { - $hashMapTerm.free(); - } """ } 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 222a1b8bc730..4cd2e788ade0 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 @@ -378,7 +378,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) val numOutput = metricTerm(ctx, "numOutputRows") val initTerm = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initRange") - val number = ctx.addMutableState(CodeGenerator.JAVA_LONG, "number") + val nextIndex = ctx.addMutableState(CodeGenerator.JAVA_LONG, "nextIndex") val value = ctx.freshName("value") val ev = ExprCode.forNonNullValue(JavaCode.variable(value, LongType)) @@ -397,7 +397,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) // within a batch, while the code in the outer loop is setting batch parameters and updating // the metrics. - // Once number == batchEnd, it's time to progress to the next batch. + // Once nextIndex == batchEnd, it's time to progress to the next batch. val batchEnd = ctx.addMutableState(CodeGenerator.JAVA_LONG, "batchEnd") // How many values should still be generated by this range operator. @@ -421,13 +421,13 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) | | $BigInt st = index.multiply(numElement).divide(numSlice).multiply(step).add(start); | if (st.compareTo($BigInt.valueOf(Long.MAX_VALUE)) > 0) { - | $number = Long.MAX_VALUE; + | $nextIndex = Long.MAX_VALUE; | } else if (st.compareTo($BigInt.valueOf(Long.MIN_VALUE)) < 0) { - | $number = Long.MIN_VALUE; + | $nextIndex = Long.MIN_VALUE; | } else { - | $number = st.longValue(); + | $nextIndex = st.longValue(); | } - | $batchEnd = $number; + | $batchEnd = $nextIndex; | | $BigInt end = index.add($BigInt.ONE).multiply(numElement).divide(numSlice) | .multiply(step).add(start); @@ -440,7 +440,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) | } | | $BigInt startToEnd = $BigInt.valueOf(partitionEnd).subtract( - | $BigInt.valueOf($number)); + | $BigInt.valueOf($nextIndex)); | $numElementsTodo = startToEnd.divide(step).longValue(); | if ($numElementsTodo < 0) { | $numElementsTodo = 0; @@ -452,12 +452,42 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) val localIdx = ctx.freshName("localIdx") val localEnd = ctx.freshName("localEnd") - val range = ctx.freshName("range") val shouldStop = if (parent.needStopCheck) { - s"if (shouldStop()) { $number = $value + ${step}L; return; }" + s"if (shouldStop()) { $nextIndex = $value + ${step}L; return; }" } else { "// shouldStop check is eliminated" } + val loopCondition = if (limitNotReachedChecks.isEmpty) { + "true" + } else { + limitNotReachedChecks.mkString(" && ") + } + + // An overview of the Range processing. + // + // For each partition, the Range task needs to produce records from partition start(inclusive) + // to end(exclusive). For better performance, we separate the partition range into batches, and + // use 2 loops to produce data. The outer while loop is used to iterate batches, and the inner + // for loop is used to iterate records inside a batch. + // + // `nextIndex` tracks the index of the next record that is going to be consumed, initialized + // with partition start. `batchEnd` tracks the end index of the current batch, initialized + // with `nextIndex`. In the outer loop, we first check if `nextIndex == batchEnd`. If it's true, + // it means the current batch is fully consumed, and we will update `batchEnd` to process the + // next batch. If `batchEnd` reaches partition end, exit the outer loop. Finally we enter the + // inner loop. Note that, when we enter inner loop, `nextIndex` must be different from + // `batchEnd`, otherwise we already exit the outer loop. + // + // The inner loop iterates from 0 to `localEnd`, which is calculated by + // `(batchEnd - nextIndex) / step`. Since `batchEnd` is increased by `nextBatchTodo * step` in + // the outer loop, and initialized with `nextIndex`, so `batchEnd - nextIndex` is always + // divisible by `step`. The `nextIndex` is increased by `step` during each iteration, and ends + // up being equal to `batchEnd` when the inner loop finishes. + // + // The inner loop can be interrupted, if the query has produced at least one result row, so that + // we don't buffer too many result rows and waste memory. It's ok to interrupt the inner loop, + // because `nextIndex` will be updated before interrupting. + s""" | // initialize Range | if (!$initTerm) { @@ -465,33 +495,30 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) | $initRangeFuncName(partitionIndex); | } | - | while (true) { - | long $range = $batchEnd - $number; - | if ($range != 0L) { - | int $localEnd = (int)($range / ${step}L); - | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { - | long $value = ((long)$localIdx * ${step}L) + $number; - | ${consume(ctx, Seq(ev))} - | $shouldStop + | while ($loopCondition) { + | if ($nextIndex == $batchEnd) { + | long $nextBatchTodo; + | if ($numElementsTodo > ${batchSize}L) { + | $nextBatchTodo = ${batchSize}L; + | $numElementsTodo -= ${batchSize}L; + | } else { + | $nextBatchTodo = $numElementsTodo; + | $numElementsTodo = 0; + | if ($nextBatchTodo == 0) break; | } - | $number = $batchEnd; + | $numOutput.add($nextBatchTodo); + | $inputMetrics.incRecordsRead($nextBatchTodo); + | $batchEnd += $nextBatchTodo * ${step}L; | } | - | $taskContext.killTaskIfInterrupted(); - | - | long $nextBatchTodo; - | if ($numElementsTodo > ${batchSize}L) { - | $nextBatchTodo = ${batchSize}L; - | $numElementsTodo -= ${batchSize}L; - | } else { - | $nextBatchTodo = $numElementsTodo; - | $numElementsTodo = 0; - | if ($nextBatchTodo == 0) break; + | int $localEnd = (int)(($batchEnd - $nextIndex) / ${step}L); + | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { + | long $value = ((long)$localIdx * ${step}L) + $nextIndex; + | ${consume(ctx, Seq(ev))} + | $shouldStop | } - | $numOutput.add($nextBatchTodo); - | $inputMetrics.incRecordsRead($nextBatchTodo); - | - | $batchEnd += $nextBatchTodo * ${step}L; + | $nextIndex = $batchEnd; + | $taskContext.killTaskIfInterrupted(); | } """.stripMargin } 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 345c9d82ca0e..dd3c154259c7 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 @@ -104,12 +104,15 @@ class FileScanRDD( val nextElement = currentIterator.next() // TODO: we should have a better separation of row based and batch based scan, so that we // don't need to run this `if` for every record. + val preNumRecordsRead = inputMetrics.recordsRead if (nextElement.isInstanceOf[ColumnarBatch]) { inputMetrics.incRecordsRead(nextElement.asInstanceOf[ColumnarBatch].numRows()) } else { inputMetrics.incRecordsRead(1) } - if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + // The records may be incremented by more than 1 at a time. + if (preNumRecordsRead / SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS != + inputMetrics.recordsRead / SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS) { updateBytesRead() } nextElement diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index dc5c2ff927e4..fe418e610da8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -315,7 +315,14 @@ object InMemoryFileIndex extends Logging { // which is very slow on some file system (RawLocalFileSystem, which is launch a // subprocess and parse the stdout). try { - val locations = fs.getFileBlockLocations(f, 0, f.getLen) + val locations = fs.getFileBlockLocations(f, 0, f.getLen).map { loc => + // Store BlockLocation objects to consume less memory + if (loc.getClass == classOf[BlockLocation]) { + loc + } else { + new BlockLocation(loc.getNames, loc.getHosts, loc.getOffset, loc.getLength) + } + } val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, f.getModificationTime, 0, null, null, null, null, f.getPath, locations) if (f.isSymlink) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala index b93f418bcb5b..0b5a719d427c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala @@ -51,11 +51,8 @@ abstract class CSVDataSource extends Serializable { conf: Configuration, file: PartitionedFile, parser: UnivocityParser, - requiredSchema: StructType, - // Actual schema of data in the csv file - dataSchema: StructType, - caseSensitive: Boolean, - columnPruning: Boolean): Iterator[InternalRow] + headerChecker: CSVHeaderChecker, + requiredSchema: StructType): Iterator[InternalRow] /** * Infers the schema from `inputPaths` files. @@ -75,48 +72,6 @@ abstract class CSVDataSource extends Serializable { sparkSession: SparkSession, inputPaths: Seq[FileStatus], parsedOptions: CSVOptions): StructType - - /** - * Generates a header from the given row which is null-safe and duplicate-safe. - */ - protected def makeSafeHeader( - row: Array[String], - caseSensitive: Boolean, - options: CSVOptions): Array[String] = { - if (options.headerFlag) { - val duplicates = { - val headerNames = row.filter(_ != null) - // scalastyle:off caselocale - .map(name => if (caseSensitive) name else name.toLowerCase) - // scalastyle:on caselocale - headerNames.diff(headerNames.distinct).distinct - } - - row.zipWithIndex.map { case (value, index) => - if (value == null || value.isEmpty || value == options.nullValue) { - // When there are empty strings or the values set in `nullValue`, put the - // index as the suffix. - s"_c$index" - // scalastyle:off caselocale - } else if (!caseSensitive && duplicates.contains(value.toLowerCase)) { - // scalastyle:on caselocale - // When there are case-insensitive duplicates, put the index as the suffix. - s"$value$index" - } else if (duplicates.contains(value)) { - // When there are duplicates, put the index as the suffix. - s"$value$index" - } else { - value - } - } - } else { - row.zipWithIndex.map { case (_, index) => - // Uses default column names, "_c#" where # is its position of fields - // when header option is disabled. - s"_c$index" - } - } - } } object CSVDataSource extends Logging { @@ -127,67 +82,6 @@ object CSVDataSource extends Logging { TextInputCSVDataSource } } - - /** - * Checks that column names in a CSV header and field names in the schema are the same - * by taking into account case sensitivity. - * - * @param schema - provided (or inferred) schema to which CSV must conform. - * @param columnNames - names of CSV columns that must be checked against to the schema. - * @param fileName - name of CSV file that are currently checked. It is used in error messages. - * @param enforceSchema - if it is `true`, column names are ignored otherwise the CSV column - * names are checked for conformance to the schema. In the case if - * the column name don't conform to the schema, an exception is thrown. - * @param caseSensitive - if it is set to `false`, comparison of column names and schema field - * names is not case sensitive. - */ - def checkHeaderColumnNames( - schema: StructType, - columnNames: Array[String], - fileName: String, - enforceSchema: Boolean, - caseSensitive: Boolean): Unit = { - if (columnNames != null) { - val fieldNames = schema.map(_.name).toIndexedSeq - val (headerLen, schemaSize) = (columnNames.size, fieldNames.length) - var errorMessage: Option[String] = None - - if (headerLen == schemaSize) { - var i = 0 - while (errorMessage.isEmpty && i < headerLen) { - var (nameInSchema, nameInHeader) = (fieldNames(i), columnNames(i)) - if (!caseSensitive) { - // scalastyle:off caselocale - nameInSchema = nameInSchema.toLowerCase - nameInHeader = nameInHeader.toLowerCase - // scalastyle:on caselocale - } - if (nameInHeader != nameInSchema) { - errorMessage = Some( - s"""|CSV header does not conform to the schema. - | Header: ${columnNames.mkString(", ")} - | Schema: ${fieldNames.mkString(", ")} - |Expected: ${fieldNames(i)} but found: ${columnNames(i)} - |CSV file: $fileName""".stripMargin) - } - i += 1 - } - } else { - errorMessage = Some( - s"""|Number of column in CSV header is not equal to number of fields in the schema: - | Header length: $headerLen, schema size: $schemaSize - |CSV file: $fileName""".stripMargin) - } - - errorMessage.foreach { msg => - if (enforceSchema) { - logWarning(msg) - } else { - throw new IllegalArgumentException(msg) - } - } - } - } } object TextInputCSVDataSource extends CSVDataSource { @@ -197,10 +91,8 @@ object TextInputCSVDataSource extends CSVDataSource { conf: Configuration, file: PartitionedFile, parser: UnivocityParser, - requiredSchema: StructType, - dataSchema: StructType, - caseSensitive: Boolean, - columnPruning: Boolean): Iterator[InternalRow] = { + headerChecker: CSVHeaderChecker, + requiredSchema: StructType): Iterator[InternalRow] = { val lines = { val linesReader = new HadoopFileLinesReader(file, conf) Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => linesReader.close())) @@ -209,25 +101,7 @@ object TextInputCSVDataSource extends CSVDataSource { } } - val hasHeader = parser.options.headerFlag && file.start == 0 - if (hasHeader) { - // Checking that column names in the header are matched to field names of the schema. - // The header will be removed from lines. - // Note: if there are only comments in the first block, the header would probably - // be not extracted. - CSVUtils.extractHeader(lines, parser.options).foreach { header => - val schema = if (columnPruning) requiredSchema else dataSchema - val columnNames = parser.tokenizer.parseLine(header) - CSVDataSource.checkHeaderColumnNames( - schema, - columnNames, - file.filePath, - parser.options.enforceSchema, - caseSensitive) - } - } - - UnivocityParser.parseIterator(lines, parser, requiredSchema) + UnivocityParser.parseIterator(lines, parser, headerChecker, requiredSchema) } override def infer( @@ -251,7 +125,7 @@ object TextInputCSVDataSource extends CSVDataSource { maybeFirstLine.map(csvParser.parseLine(_)) match { case Some(firstRow) if firstRow != null => val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis - val header = makeSafeHeader(firstRow, caseSensitive, parsedOptions) + val header = CSVUtils.makeSafeHeader(firstRow, caseSensitive, parsedOptions) val sampled: Dataset[String] = CSVUtils.sample(csv, parsedOptions) val tokenRDD = sampled.rdd.mapPartitions { iter => val filteredLines = CSVUtils.filterCommentAndEmpty(iter, parsedOptions) @@ -298,26 +172,13 @@ object MultiLineCSVDataSource extends CSVDataSource { conf: Configuration, file: PartitionedFile, parser: UnivocityParser, - requiredSchema: StructType, - dataSchema: StructType, - caseSensitive: Boolean, - columnPruning: Boolean): Iterator[InternalRow] = { - def checkHeader(header: Array[String]): Unit = { - val schema = if (columnPruning) requiredSchema else dataSchema - CSVDataSource.checkHeaderColumnNames( - schema, - header, - file.filePath, - parser.options.enforceSchema, - caseSensitive) - } - + headerChecker: CSVHeaderChecker, + requiredSchema: StructType): Iterator[InternalRow] = { UnivocityParser.parseStream( CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath))), - parser.options.headerFlag, parser, - requiredSchema, - checkHeader) + headerChecker, + requiredSchema) } override def infer( @@ -334,7 +195,7 @@ object MultiLineCSVDataSource extends CSVDataSource { }.take(1).headOption match { case Some(firstRow) => val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis - val header = makeSafeHeader(firstRow, caseSensitive, parsedOptions) + val header = CSVUtils.makeSafeHeader(firstRow, caseSensitive, parsedOptions) val tokenRDD = csv.flatMap { lines => UnivocityParser.tokenizeStream( CodecStreams.createInputStreamWithCloseResource( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 9aad0bd55e73..3de1c2d955d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -130,7 +130,6 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { "df.filter($\"_corrupt_record\".isNotNull).count()." ) } - val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis val columnPruning = sparkSession.sessionState.conf.csvColumnPruning (file: PartitionedFile) => { @@ -139,14 +138,16 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { StructType(dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)), StructType(requiredSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)), parsedOptions) + val schema = if (columnPruning) requiredSchema else dataSchema + val isStartOfFile = file.start == 0 + val headerChecker = new CSVHeaderChecker( + schema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) CSVDataSource(parsedOptions).readFile( conf, file, parser, - requiredSchema, - dataSchema, - caseSensitive, - columnPruning) + headerChecker, + requiredSchema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVHeaderChecker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVHeaderChecker.scala new file mode 100644 index 000000000000..558ee91c419b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVHeaderChecker.scala @@ -0,0 +1,131 @@ +/* + * 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.csv + +import com.univocity.parsers.csv.CsvParser + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +/** + * Checks that column names in a CSV header and field names in the schema are the same + * by taking into account case sensitivity. + * + * @param schema provided (or inferred) schema to which CSV must conform. + * @param options parsed CSV options. + * @param source name of CSV source that are currently checked. It is used in error messages. + * @param isStartOfFile indicates if the currently processing partition is the start of the file. + * if unknown or not applicable (for instance when the input is a dataset), + * can be omitted. + */ +class CSVHeaderChecker( + schema: StructType, + options: CSVOptions, + source: String, + isStartOfFile: Boolean = false) extends Logging { + + // Indicates if it is set to `false`, comparison of column names and schema field + // names is not case sensitive. + private val caseSensitive = SQLConf.get.caseSensitiveAnalysis + + // Indicates if it is `true`, column names are ignored otherwise the CSV column + // names are checked for conformance to the schema. In the case if + // the column name don't conform to the schema, an exception is thrown. + private val enforceSchema = options.enforceSchema + + /** + * Checks that column names in a CSV header and field names in the schema are the same + * by taking into account case sensitivity. + * + * @param columnNames names of CSV columns that must be checked against to the schema. + */ + private def checkHeaderColumnNames(columnNames: Array[String]): Unit = { + if (columnNames != null) { + val fieldNames = schema.map(_.name).toIndexedSeq + val (headerLen, schemaSize) = (columnNames.size, fieldNames.length) + var errorMessage: Option[String] = None + + if (headerLen == schemaSize) { + var i = 0 + while (errorMessage.isEmpty && i < headerLen) { + var (nameInSchema, nameInHeader) = (fieldNames(i), columnNames(i)) + if (!caseSensitive) { + // scalastyle:off caselocale + nameInSchema = nameInSchema.toLowerCase + nameInHeader = nameInHeader.toLowerCase + // scalastyle:on caselocale + } + if (nameInHeader != nameInSchema) { + errorMessage = Some( + s"""|CSV header does not conform to the schema. + | Header: ${columnNames.mkString(", ")} + | Schema: ${fieldNames.mkString(", ")} + |Expected: ${fieldNames(i)} but found: ${columnNames(i)} + |$source""".stripMargin) + } + i += 1 + } + } else { + errorMessage = Some( + s"""|Number of column in CSV header is not equal to number of fields in the schema: + | Header length: $headerLen, schema size: $schemaSize + |$source""".stripMargin) + } + + errorMessage.foreach { msg => + if (enforceSchema) { + logWarning(msg) + } else { + throw new IllegalArgumentException(msg) + } + } + } + } + + // This is currently only used to parse CSV from Dataset[String]. + def checkHeaderColumnNames(line: String): Unit = { + if (options.headerFlag) { + val parser = new CsvParser(options.asParserSettings) + checkHeaderColumnNames(parser.parseLine(line)) + } + } + + // This is currently only used to parse CSV with multiLine mode. + private[csv] def checkHeaderColumnNames(tokenizer: CsvParser): Unit = { + assert(options.multiLine, "This method should be executed with multiLine.") + if (options.headerFlag) { + val firstRecord = tokenizer.parseNext() + checkHeaderColumnNames(firstRecord) + } + } + + // This is currently only used to parse CSV with non-multiLine mode. + private[csv] def checkHeaderColumnNames(lines: Iterator[String], tokenizer: CsvParser): Unit = { + assert(!options.multiLine, "This method should not be executed with multiline.") + // Checking that column names in the header are matched to field names of the schema. + // The header will be removed from lines. + // Note: if there are only comments in the first block, the header would probably + // be not extracted. + if (options.headerFlag && isStartOfFile) { + CSVUtils.extractHeader(lines, options).foreach { header => + checkHeaderColumnNames(tokenizer.parseLine(header)) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index a585cbed2551..3596ff105fd7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -70,7 +70,7 @@ private[csv] object CSVInferSchema { def mergeRowTypes(first: Array[DataType], second: Array[DataType]): Array[DataType] = { first.zipAll(second, NullType, NullType).map { case (a, b) => - findTightestCommonType(a, b).getOrElse(NullType) + compatibleType(a, b).getOrElse(NullType) } } @@ -88,7 +88,7 @@ private[csv] object CSVInferSchema { case LongType => tryParseLong(field, options) case _: DecimalType => // DecimalTypes have different precisions and scales, so we try to find the common type. - findTightestCommonType(typeSoFar, tryParseDecimal(field, options)).getOrElse(StringType) + compatibleType(typeSoFar, tryParseDecimal(field, options)).getOrElse(StringType) case DoubleType => tryParseDouble(field, options) case TimestampType => tryParseTimestamp(field, options) case BooleanType => tryParseBoolean(field, options) @@ -172,35 +172,27 @@ private[csv] object CSVInferSchema { StringType } - private val numericPrecedence: IndexedSeq[DataType] = TypeCoercion.numericPrecedence + /** + * Returns the common data type given two input data types so that the return type + * is compatible with both input data types. + */ + private def compatibleType(t1: DataType, t2: DataType): Option[DataType] = { + TypeCoercion.findTightestCommonType(t1, t2).orElse(findCompatibleTypeForCSV(t1, t2)) + } /** - * Copied from internal Spark api - * [[org.apache.spark.sql.catalyst.analysis.TypeCoercion]] + * The following pattern matching represents additional type promotion rules that + * are CSV specific. */ - val findTightestCommonType: (DataType, DataType) => Option[DataType] = { - case (t1, t2) if t1 == t2 => Some(t1) - case (NullType, t1) => Some(t1) - case (t1, NullType) => Some(t1) + private val findCompatibleTypeForCSV: (DataType, DataType) => Option[DataType] = { case (StringType, t2) => Some(StringType) case (t1, StringType) => Some(StringType) - // Promote numeric types to the highest of the two and all numeric types to unlimited decimal - case (t1, t2) if Seq(t1, t2).forall(numericPrecedence.contains) => - val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) - Some(numericPrecedence(index)) - - // These two cases below deal with when `DecimalType` is larger than `IntegralType`. - case (t1: IntegralType, t2: DecimalType) if t2.isWiderThan(t1) => - Some(t2) - case (t1: DecimalType, t2: IntegralType) if t1.isWiderThan(t2) => - Some(t1) - // These two cases below deal with when `IntegralType` is larger than `DecimalType`. case (t1: IntegralType, t2: DecimalType) => - findTightestCommonType(DecimalType.forType(t1), t2) + compatibleType(DecimalType.forType(t1), t2) case (t1: DecimalType, t2: IntegralType) => - findTightestCommonType(t1, DecimalType.forType(t2)) + compatibleType(t1, DecimalType.forType(t2)) // Double support larger range than fixed decimal, DecimalType.Maximum should be enough // in most case, also have better precision. @@ -216,7 +208,6 @@ private[csv] object CSVInferSchema { } else { Some(DecimalType(range + scale, scale)) } - case _ => None } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala index 7ce65fa89b02..b912f8add3af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.datasources.csv import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dataset import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types._ object CSVUtils { /** @@ -90,6 +89,49 @@ object CSVUtils { None } } + + /** + * Generates a header from the given row which is null-safe and duplicate-safe. + */ + def makeSafeHeader( + row: Array[String], + caseSensitive: Boolean, + options: CSVOptions): Array[String] = { + if (options.headerFlag) { + val duplicates = { + val headerNames = row.filter(_ != null) + // scalastyle:off caselocale + .map(name => if (caseSensitive) name else name.toLowerCase) + // scalastyle:on caselocale + headerNames.diff(headerNames.distinct).distinct + } + + row.zipWithIndex.map { case (value, index) => + if (value == null || value.isEmpty || value == options.nullValue) { + // When there are empty strings or the values set in `nullValue`, put the + // index as the suffix. + s"_c$index" + // scalastyle:off caselocale + } else if (!caseSensitive && duplicates.contains(value.toLowerCase)) { + // scalastyle:on caselocale + // When there are case-insensitive duplicates, put the index as the suffix. + s"$value$index" + } else if (duplicates.contains(value)) { + // When there are duplicates, put the index as the suffix. + s"$value$index" + } else { + value + } + } + } else { + row.zipWithIndex.map { case (_, index) => + // Uses default column names, "_c#" where # is its position of fields + // when header option is disabled. + s"_c$index" + } + } + } + /** * Helper method that converts string representation of a character to actual character. * It handles some Java escaped strings and throws exception if given string is longer than one diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala index 9088d43905e2..fbd19c6e677e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala @@ -273,7 +273,10 @@ private[csv] object UnivocityParser { inputStream: InputStream, shouldDropHeader: Boolean, tokenizer: CsvParser): Iterator[Array[String]] = { - convertStream(inputStream, shouldDropHeader, tokenizer)(tokens => tokens) + val handleHeader: () => Unit = + () => if (shouldDropHeader) tokenizer.parseNext + + convertStream(inputStream, tokenizer, handleHeader)(tokens => tokens) } /** @@ -281,10 +284,9 @@ private[csv] object UnivocityParser { */ def parseStream( inputStream: InputStream, - shouldDropHeader: Boolean, parser: UnivocityParser, - schema: StructType, - checkHeader: Array[String] => Unit): Iterator[InternalRow] = { + headerChecker: CSVHeaderChecker, + schema: StructType): Iterator[InternalRow] = { val tokenizer = parser.tokenizer val safeParser = new FailureSafeParser[Array[String]]( input => Seq(parser.convert(input)), @@ -292,25 +294,26 @@ private[csv] object UnivocityParser { schema, parser.options.columnNameOfCorruptRecord, parser.options.multiLine) - convertStream(inputStream, shouldDropHeader, tokenizer, checkHeader) { tokens => + + val handleHeader: () => Unit = + () => headerChecker.checkHeaderColumnNames(tokenizer) + + convertStream(inputStream, tokenizer, handleHeader) { tokens => safeParser.parse(tokens) }.flatten } private def convertStream[T]( inputStream: InputStream, - shouldDropHeader: Boolean, tokenizer: CsvParser, - checkHeader: Array[String] => Unit = _ => ())( + handleHeader: () => Unit)( convert: Array[String] => T) = new Iterator[T] { tokenizer.beginParsing(inputStream) - private var nextRecord = { - if (shouldDropHeader) { - val firstRecord = tokenizer.parseNext() - checkHeader(firstRecord) - } - tokenizer.parseNext() - } + + // We can handle header here since here the stream is open. + handleHeader() + + private var nextRecord = tokenizer.parseNext() override def hasNext: Boolean = nextRecord != null @@ -330,7 +333,10 @@ private[csv] object UnivocityParser { def parseIterator( lines: Iterator[String], parser: UnivocityParser, + headerChecker: CSVHeaderChecker, schema: StructType): Iterator[InternalRow] = { + headerChecker.checkHeaderColumnNames(lines, parser.tokenizer) + val options = parser.options val filteredLines: Iterator[String] = CSVUtils.filterCommentAndEmpty(lines, options) 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 dbafc468c6c4..2b17b479432f 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 @@ -138,6 +138,23 @@ private[sql] object OrcFilters { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { + createBuilder(dataTypeMap, expression, builder, canPartialPushDownConjuncts = true) + } + + /** + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input filter predicates. + * @param builder the input SearchArgument.Builder. + * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed + * down safely. Pushing ONLY one side of AND down is safe to + * do at the top level or none of its ancestors is NOT and OR. + * @return the builder so far. + */ + private def createBuilder( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder, + canPartialPushDownConjuncts: Boolean): Option[Builder] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -145,32 +162,52 @@ private[sql] object OrcFilters { expression match { case And(left, right) => - // At here, it is not safe to just convert one side if we do not understand the - // other side. Here is an example used to explain the reason. + // At here, it is not safe to just convert one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to // convert b in ('1'). If we only convert a = 2, we will end up with a filter // NOT(a = 2), which will generate wrong results. - // Pushing one side of AND down is only safe to do at the top level. - // You can see ParquetRelation's initializeLocalJobFunc method as an example. - for { - _ <- buildSearchArgument(dataTypeMap, left, newBuilder) - _ <- buildSearchArgument(dataTypeMap, right, newBuilder) - lhs <- buildSearchArgument(dataTypeMap, left, builder.startAnd()) - rhs <- buildSearchArgument(dataTypeMap, right, lhs) - } yield rhs.end() + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate + // can be safely removed. + val leftBuilderOption = + createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) + val rightBuilderOption = + createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) + (leftBuilderOption, rightBuilderOption) match { + case (Some(_), Some(_)) => + for { + lhs <- createBuilder(dataTypeMap, left, + builder.startAnd(), canPartialPushDownConjuncts) + rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) + } yield rhs.end() + + case (Some(_), None) if canPartialPushDownConjuncts => + createBuilder(dataTypeMap, left, builder, canPartialPushDownConjuncts) + + case (None, Some(_)) if canPartialPushDownConjuncts => + createBuilder(dataTypeMap, right, builder, canPartialPushDownConjuncts) + + case _ => None + } case Or(left, right) => for { - _ <- buildSearchArgument(dataTypeMap, left, newBuilder) - _ <- buildSearchArgument(dataTypeMap, right, newBuilder) - lhs <- buildSearchArgument(dataTypeMap, left, builder.startOr()) - rhs <- buildSearchArgument(dataTypeMap, right, lhs) + _ <- createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts = false) + _ <- createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts = false) + lhs <- createBuilder(dataTypeMap, left, + builder.startOr(), canPartialPushDownConjuncts = false) + rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts = false) } yield rhs.end() case Not(child) => for { - _ <- buildSearchArgument(dataTypeMap, child, newBuilder) - negate <- buildSearchArgument(dataTypeMap, child, builder.startNot()) + _ <- createBuilder(dataTypeMap, child, newBuilder, canPartialPushDownConjuncts = false) + negate <- createBuilder(dataTypeMap, + child, builder.startNot(), canPartialPushDownConjuncts = false) } yield negate.end() // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 44a0d209e6e6..21ab9c78e53d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -394,13 +394,22 @@ private[parquet] class ParquetFilters( */ def createFilter(schema: MessageType, predicate: sources.Filter): Option[FilterPredicate] = { val nameToParquetField = getFieldMap(schema) - createFilterHelper(nameToParquetField, predicate, canRemoveOneSideInAnd = true) + createFilterHelper(nameToParquetField, predicate, canPartialPushDownConjuncts = true) } + /** + * @param nameToParquetField a map from the field name to its field name and data type. + * This only includes the root fields whose types are primitive types. + * @param predicate the input filter predicates. Not all the predicates can be pushed down. + * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed + * down safely. Pushing ONLY one side of AND down is safe to + * do at the top level or none of its ancestors is NOT and OR. + * @return the Parquet-native filter predicates that are eligible for pushdown. + */ private def createFilterHelper( nameToParquetField: Map[String, ParquetField], predicate: sources.Filter, - canRemoveOneSideInAnd: Boolean): Option[FilterPredicate] = { + canPartialPushDownConjuncts: Boolean): Option[FilterPredicate] = { // Decimal type must make sure that filter value's scale matched the file. // If doesn't matched, which would cause data corruption. def isDecimalMatched(value: Any, decimalMeta: DecimalMetadata): Boolean = value match { @@ -505,24 +514,28 @@ private[parquet] class ParquetFilters( // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate // can be safely removed. - val lhsFilterOption = createFilterHelper(nameToParquetField, lhs, canRemoveOneSideInAnd) - val rhsFilterOption = createFilterHelper(nameToParquetField, rhs, canRemoveOneSideInAnd) + val lhsFilterOption = + createFilterHelper(nameToParquetField, lhs, canPartialPushDownConjuncts) + val rhsFilterOption = + createFilterHelper(nameToParquetField, rhs, canPartialPushDownConjuncts) (lhsFilterOption, rhsFilterOption) match { case (Some(lhsFilter), Some(rhsFilter)) => Some(FilterApi.and(lhsFilter, rhsFilter)) - case (Some(lhsFilter), None) if canRemoveOneSideInAnd => Some(lhsFilter) - case (None, Some(rhsFilter)) if canRemoveOneSideInAnd => Some(rhsFilter) + case (Some(lhsFilter), None) if canPartialPushDownConjuncts => Some(lhsFilter) + case (None, Some(rhsFilter)) if canPartialPushDownConjuncts => Some(rhsFilter) case _ => None } case sources.Or(lhs, rhs) => for { - lhsFilter <- createFilterHelper(nameToParquetField, lhs, canRemoveOneSideInAnd = false) - rhsFilter <- createFilterHelper(nameToParquetField, rhs, canRemoveOneSideInAnd = false) + lhsFilter <- + createFilterHelper(nameToParquetField, lhs, canPartialPushDownConjuncts = false) + rhsFilter <- + createFilterHelper(nameToParquetField, rhs, canPartialPushDownConjuncts = false) } yield FilterApi.or(lhsFilter, rhsFilter) case sources.Not(pred) => - createFilterHelper(nameToParquetField, pred, canRemoveOneSideInAnd = false) + createFilterHelper(nameToParquetField, pred, canPartialPushDownConjuncts = false) .map(FilterApi.not) case sources.In(name, values) if canMakeFilterOn(name, values.head) 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 f4b9d132122e..d7d3f6d6078b 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 @@ -494,7 +494,7 @@ case class SortMergeJoinExec( | $leftRow = null; | } else { | $matches.add((UnsafeRow) $rightRow); - | $rightRow = null;; + | $rightRow = null; | } | } while ($leftRow != null); | } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 66bcda891373..9bfe1a79fc1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -46,6 +46,15 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode } } +object BaseLimitExec { + private val curId = new java.util.concurrent.atomic.AtomicInteger() + + def newLimitCountTerm(): String = { + val id = curId.getAndIncrement() + s"_limit_counter_$id" + } +} + /** * Helper trait which defines methods that are shared by both * [[LocalLimitExec]] and [[GlobalLimitExec]]. @@ -66,27 +75,25 @@ trait BaseLimitExec extends UnaryExecNode with CodegenSupport { // to the parent operator. override def usedInputs: AttributeSet = AttributeSet.empty + private lazy val countTerm = BaseLimitExec.newLimitCountTerm() + + override lazy val limitNotReachedChecks: Seq[String] = { + s"$countTerm < $limit" +: super.limitNotReachedChecks + } + protected override def doProduce(ctx: CodegenContext): String = { child.asInstanceOf[CodegenSupport].produce(ctx, this) } override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { - val stopEarly = - ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "stopEarly") // init as stopEarly = false - - ctx.addNewFunction("stopEarly", s""" - @Override - protected boolean stopEarly() { - return $stopEarly; - } - """, inlineToOuterClass = true) - val countTerm = ctx.addMutableState(CodeGenerator.JAVA_INT, "count") // init as count = 0 + // The counter name is already obtained by the upstream operators via `limitNotReachedChecks`. + // Here we have to inline it to not change its name. This is fine as we won't have many limit + // operators in one query. + ctx.addMutableState(CodeGenerator.JAVA_INT, countTerm, forceInline = true, useFreshName = false) s""" | if ($countTerm < $limit) { | $countTerm += 1; | ${consume(ctx, input)} - | } else { - | $stopEarly = true; | } """.stripMargin } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 367ac66dd77f..4247d3110f1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2546,15 +2546,39 @@ object functions { def soundex(e: Column): Column = withExpr { SoundEx(e.expr) } /** - * Splits str around pattern (pattern is a regular expression). + * Splits str around matches of the given regex. * - * @note Pattern is a string representation of the regular expression. + * @param str a string expression to split + * @param regex a string representing a regular expression. The regex string should be + * a Java regular expression. * * @group string_funcs * @since 1.5.0 */ - def split(str: Column, pattern: String): Column = withExpr { - StringSplit(str.expr, lit(pattern).expr) + def split(str: Column, regex: String): Column = withExpr { + StringSplit(str.expr, Literal(regex), Literal(-1)) + } + + /** + * Splits str around matches of the given regex. + * + * @param str a string expression to split + * @param regex a string representing a regular expression. The regex string should be + * a Java regular expression. + * @param limit an integer expression which controls the number of times the regex is applied. + *
      + *
    • limit greater than 0: The resulting array's length will not be more than limit, + * and the resulting array's last entry will contain all input beyond the last + * matched regex.
    • + *
    • limit less than or equal to 0: `regex` will be applied as many times as + * possible, and the resulting array can be of any size.
    • + *
    + * + * @group string_funcs + * @since 3.0.0 + */ + def split(str: Column, regex: String, limit: Int): Column = withExpr { + StringSplit(str.expr, Literal(regex), Literal(limit)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index e9a15214d952..b23e86a78645 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -380,7 +380,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * @since 2.4.0 */ @InterfaceStability.Evolving - def foreachBatch(function: VoidFunction2[Dataset[T], Long]): DataStreamWriter[T] = { + def foreachBatch(function: VoidFunction2[Dataset[T], java.lang.Long]): DataStreamWriter[T] = { foreachBatch((batchDs: Dataset[T], batchId: Long) => function.call(batchDs, batchId)) } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 00f41d6484af..a05afa4f6ba3 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -134,6 +134,8 @@ public static class Bean implements Serializable { private Map c = ImmutableMap.of("hello", new int[] { 1, 2 }); private List d = Arrays.asList("floppy", "disk"); private BigInteger e = new BigInteger("1234567"); + private NestedBean f = new NestedBean(); + private NestedBean g = null; public double getA() { return a; @@ -152,6 +154,22 @@ public List getD() { } public BigInteger getE() { return e; } + + public NestedBean getF() { + return f; + } + + public NestedBean getG() { + return g; + } + + public static class NestedBean implements Serializable { + private int a = 1; + + public int getA() { + return a; + } + } } void validateDataFrameWithBeans(Bean bean, Dataset df) { @@ -171,7 +189,14 @@ void validateDataFrameWithBeans(Bean bean, Dataset df) { schema.apply("d")); Assert.assertEquals(new StructField("e", DataTypes.createDecimalType(38,0), true, Metadata.empty()), schema.apply("e")); - Row first = df.select("a", "b", "c", "d", "e").first(); + StructType nestedBeanType = + DataTypes.createStructType(Collections.singletonList(new StructField( + "a", IntegerType$.MODULE$, false, Metadata.empty()))); + Assert.assertEquals(new StructField("f", nestedBeanType, true, Metadata.empty()), + schema.apply("f")); + Assert.assertEquals(new StructField("g", nestedBeanType, true, Metadata.empty()), + schema.apply("g")); + Row first = df.select("a", "b", "c", "d", "e", "f", "g").first(); Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); // Now Java lists and maps are converted to Scala Seq's and Map's. Once we get a Seq below, // verify that it has the expected length, and contains expected elements. @@ -192,6 +217,9 @@ void validateDataFrameWithBeans(Bean bean, Dataset df) { } // Java.math.BigInteger is equivalent to Spark Decimal(38,0) Assert.assertEquals(new BigDecimal(bean.getE()), first.getDecimal(4)); + Row nested = first.getStruct(5); + Assert.assertEquals(bean.getF().getA(), nested.getInt(0)); + Assert.assertTrue(first.isNullAt(6)); } @Test diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java new file mode 100644 index 000000000000..48cdb2642d83 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java @@ -0,0 +1,89 @@ +/* + * 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 test.org.apache.spark.sql.streaming; + +import java.io.File; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.function.VoidFunction2; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.ForeachWriter; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.test.TestSparkSession; +import org.apache.spark.util.Utils; + +public class JavaDataStreamReaderWriterSuite { + private SparkSession spark; + private String input; + + @Before + public void setUp() { + spark = new TestSparkSession(); + input = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "input").toString(); + } + + @After + public void tearDown() { + try { + Utils.deleteRecursively(new File(input)); + } finally { + spark.stop(); + spark = null; + } + } + + @Test + public void testForeachBatchAPI() { + StreamingQuery query = spark + .readStream() + .textFile(input) + .writeStream() + .foreachBatch(new VoidFunction2, Long>() { + @Override + public void call(Dataset v1, Long v2) throws Exception {} + }) + .start(); + query.stop(); + } + + @Test + public void testForeachAPI() { + StreamingQuery query = spark + .readStream() + .textFile(input) + .writeStream() + .foreach(new ForeachWriter() { + @Override + public boolean open(long partitionId, long epochId) { + return true; + } + + @Override + public void process(String value) {} + + @Override + public void close(Throwable errorOrNull) {} + }) + .start(); + query.stop(); + } +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 2c18d6aaabdb..433db7152743 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -73,3 +73,10 @@ where b.z != b.z; -- SPARK-24369 multiple distinct aggregations having the same argument set SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; diff --git a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql index 4113734e1707..2effb43183d7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql @@ -46,4 +46,8 @@ FROM ( encode(string(id + 2), 'utf-8') col3, encode(string(id + 3), 'utf-8') col4 FROM range(10) -) +); + +-- split function +SELECT split('aa1cc2ee3', '[1-9]+'); +SELECT split('aa1cc2ee3', '[1-9]+', 2); diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 581aa1754ce1..f9d1ee8a6bcd 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 27 +-- Number of queries: 30 -- !query 0 @@ -250,3 +250,28 @@ SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) struct -- !query 26 output 1.0 1.0 3 + + +-- !query 27 +SELECT 1 FROM range(10) HAVING true +-- !query 27 schema +struct<1:int> +-- !query 27 output +1 + + +-- !query 28 +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query 28 schema +struct<1:int> +-- !query 28 output +1 + + +-- !query 29 +SELECT id FROM range(10) HAVING id > 0 +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and '`id`' is not an aggregate function. Wrap '()' in windowing function(s) or wrap '`id`' in first() (or first_value) if you don't care which value you get.; diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index 7b3dc8438888..e8f2e0a81455 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 15 +-- Number of queries: 17 -- !query 0 @@ -161,3 +161,19 @@ struct == Physical Plan == *Project [concat(cast(id#xL as string), cast(encode(cast((id#xL + 2) as string), utf-8) as string), cast(encode(cast((id#xL + 3) as string), utf-8) as string)) AS col#x] +- *Range (0, 10, step=1, splits=2) + + +-- !query 15 +SELECT split('aa1cc2ee3', '[1-9]+') +-- !query 15 schema +struct> +-- !query 15 output +["aa","cc","ee",""] + + +-- !query 16 +SELECT split('aa1cc2ee3', '[1-9]+', 2) +-- !query 16 schema +struct> +-- !query 16 output +["aa","cc2ee3"] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 5c6a021d5b76..fef6ddd0b93c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -127,8 +127,8 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits } test("cache UDF result correctly") { - val expensiveUDF = udf({x: Int => Thread.sleep(5000); x}) - val df = spark.range(0, 10).toDF("a").withColumn("b", expensiveUDF($"a")) + val expensiveUDF = udf({x: Int => Thread.sleep(2000); x}) + val df = spark.range(0, 2).toDF("a").repartition(1).withColumn("b", expensiveUDF($"a")) val df2 = df.agg(sum(df("b"))) df.cache() @@ -136,7 +136,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits assertCached(df2) // udf has been evaluated during caching, and thus should not be re-evaluated here - failAfter(3 seconds) { + failAfter(2 seconds) { df2.collect() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 3d76b9ac33e5..bb19fde2b2b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -329,16 +329,52 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { Row(" ")) } - test("string split function") { - val df = Seq(("aa2bb3cc", "[1-9]+")).toDF("a", "b") + test("string split function with no limit") { + val df = Seq(("aa2bb3cc4", "[1-9]+")).toDF("a", "b") checkAnswer( df.select(split($"a", "[1-9]+")), - Row(Seq("aa", "bb", "cc"))) + Row(Seq("aa", "bb", "cc", ""))) checkAnswer( df.selectExpr("split(a, '[1-9]+')"), - Row(Seq("aa", "bb", "cc"))) + Row(Seq("aa", "bb", "cc", ""))) + } + + test("string split function with limit explicitly set to 0") { + val df = Seq(("aa2bb3cc4", "[1-9]+")).toDF("a", "b") + + checkAnswer( + df.select(split($"a", "[1-9]+", 0)), + Row(Seq("aa", "bb", "cc", ""))) + + checkAnswer( + df.selectExpr("split(a, '[1-9]+', 0)"), + Row(Seq("aa", "bb", "cc", ""))) + } + + test("string split function with positive limit") { + val df = Seq(("aa2bb3cc4", "[1-9]+")).toDF("a", "b") + + checkAnswer( + df.select(split($"a", "[1-9]+", 2)), + Row(Seq("aa", "bb3cc4"))) + + checkAnswer( + df.selectExpr("split(a, '[1-9]+', 2)"), + Row(Seq("aa", "bb3cc4"))) + } + + test("string split function with negative limit") { + val df = Seq(("aa2bb3cc4", "[1-9]+")).toDF("a", "b") + + checkAnswer( + df.select(split($"a", "[1-9]+", -2)), + Row(Seq("aa", "bb", "cc", ""))) + + checkAnswer( + df.selectExpr("split(a, '[1-9]+', -2)"), + Row(Seq("aa", "bb", "cc", ""))) } test("string / binary length function") { 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 51a7f9f1ef09..a1e7f9e36f4b 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 @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import scala.util.Random import org.apache.spark.SparkConf -import org.apache.spark.benchmark.Benchmark +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.SQLHelper @@ -34,10 +34,16 @@ import org.apache.spark.sql.vectorized.ColumnVector /** * Benchmark to measure data source read performance. - * To run this: - * spark-submit --class + * To run this benchmark: + * {{{ + * 1. without sbt: bin/spark-submit --class + * --jars , + * 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/DataSourceReadBenchmark-results.txt". + * }}} */ -object DataSourceReadBenchmark extends SQLHelper { +object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { val conf = new SparkConf() .setAppName("DataSourceReadBenchmark") // Since `spark.master` always exists, overrides this value @@ -93,11 +99,16 @@ object DataSourceReadBenchmark extends SQLHelper { def numericScanBenchmark(values: Int, dataType: DataType): Unit = { // Benchmarks running through spark sql. - val sqlBenchmark = new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values) + val sqlBenchmark = new Benchmark( + s"SQL Single ${dataType.sql} Column Scan", + values, + output = output) // Benchmarks driving reader component directly. val parquetReaderBenchmark = new Benchmark( - s"Parquet Reader Single ${dataType.sql} Column Scan", values) + s"Parquet Reader Single ${dataType.sql} Column Scan", + values, + output = output) withTempPath { dir => withTempTable("t1", "csvTable", "jsonTable", "parquetTable", "orcTable") { @@ -140,74 +151,6 @@ object DataSourceReadBenchmark extends SQLHelper { } } - /* - OpenJDK 64-Bit Server VM 1.8.0_171-b10 on Linux 4.14.33-51.37.amzn1.x86_64 - Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz - SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 22964 / 23096 0.7 1460.0 1.0X - SQL Json 8469 / 8593 1.9 538.4 2.7X - SQL Parquet Vectorized 164 / 177 95.8 10.4 139.9X - SQL Parquet MR 1687 / 1706 9.3 107.2 13.6X - SQL ORC Vectorized 191 / 197 82.3 12.2 120.2X - SQL ORC Vectorized with copy 215 / 219 73.2 13.7 106.9X - SQL ORC MR 1392 / 1412 11.3 88.5 16.5X - - - SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 24090 / 24097 0.7 1531.6 1.0X - SQL Json 8791 / 8813 1.8 558.9 2.7X - SQL Parquet Vectorized 204 / 212 77.0 13.0 117.9X - SQL Parquet MR 1813 / 1850 8.7 115.3 13.3X - SQL ORC Vectorized 226 / 230 69.7 14.4 106.7X - SQL ORC Vectorized with copy 295 / 298 53.3 18.8 81.6X - SQL ORC MR 1526 / 1549 10.3 97.1 15.8X - - - SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 25637 / 25791 0.6 1629.9 1.0X - SQL Json 9532 / 9570 1.7 606.0 2.7X - SQL Parquet Vectorized 181 / 191 86.8 11.5 141.5X - SQL Parquet MR 2210 / 2227 7.1 140.5 11.6X - SQL ORC Vectorized 309 / 317 50.9 19.6 83.0X - SQL ORC Vectorized with copy 316 / 322 49.8 20.1 81.2X - SQL ORC MR 1650 / 1680 9.5 104.9 15.5X - - - SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 31617 / 31764 0.5 2010.1 1.0X - SQL Json 12440 / 12451 1.3 790.9 2.5X - SQL Parquet Vectorized 284 / 315 55.4 18.0 111.4X - SQL Parquet MR 2382 / 2390 6.6 151.5 13.3X - SQL ORC Vectorized 398 / 403 39.5 25.3 79.5X - SQL ORC Vectorized with copy 410 / 413 38.3 26.1 77.1X - SQL ORC MR 1783 / 1813 8.8 113.4 17.7X - - - SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 26679 / 26742 0.6 1696.2 1.0X - SQL Json 12490 / 12541 1.3 794.1 2.1X - SQL Parquet Vectorized 174 / 183 90.4 11.1 153.3X - SQL Parquet MR 2201 / 2223 7.1 140.0 12.1X - SQL ORC Vectorized 415 / 429 37.9 26.4 64.3X - SQL ORC Vectorized with copy 422 / 428 37.2 26.9 63.2X - SQL ORC MR 1767 / 1773 8.9 112.3 15.1X - - - SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 34223 / 34324 0.5 2175.8 1.0X - SQL Json 17784 / 17785 0.9 1130.7 1.9X - SQL Parquet Vectorized 277 / 283 56.7 17.6 123.4X - SQL Parquet MR 2356 / 2386 6.7 149.8 14.5X - SQL ORC Vectorized 533 / 536 29.5 33.9 64.2X - SQL ORC Vectorized with copy 541 / 546 29.1 34.4 63.3X - SQL ORC MR 2166 / 2177 7.3 137.7 15.8X - */ sqlBenchmark.run() // Driving the parquet reader in batch mode directly. @@ -279,51 +222,13 @@ object DataSourceReadBenchmark extends SQLHelper { } } - /* - OpenJDK 64-Bit Server VM 1.8.0_171-b10 on Linux 4.14.33-51.37.amzn1.x86_64 - Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz - Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - ParquetReader Vectorized 198 / 202 79.4 12.6 1.0X - ParquetReader Vectorized -> Row 119 / 121 132.3 7.6 1.7X - - - Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - ParquetReader Vectorized 282 / 287 55.8 17.9 1.0X - ParquetReader Vectorized -> Row 246 / 247 64.0 15.6 1.1X - - - Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - ParquetReader Vectorized 258 / 262 60.9 16.4 1.0X - ParquetReader Vectorized -> Row 259 / 260 60.8 16.5 1.0X - - - Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - ParquetReader Vectorized 361 / 369 43.6 23.0 1.0X - ParquetReader Vectorized -> Row 361 / 371 43.6 22.9 1.0X - - - Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - ParquetReader Vectorized 253 / 261 62.2 16.1 1.0X - ParquetReader Vectorized -> Row 254 / 256 61.9 16.2 1.0X - - - Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - ParquetReader Vectorized 357 / 364 44.0 22.7 1.0X - ParquetReader Vectorized -> Row 358 / 366 44.0 22.7 1.0X - */ parquetReaderBenchmark.run() } } } def intStringScanBenchmark(values: Int): Unit = { - val benchmark = new Benchmark("Int and String Scan", values) + val benchmark = new Benchmark("Int and String Scan", values, output = output) withTempPath { dir => withTempTable("t1", "csvTable", "jsonTable", "parquetTable", "orcTable") { @@ -368,26 +273,13 @@ object DataSourceReadBenchmark extends SQLHelper { } } - /* - OpenJDK 64-Bit Server VM 1.8.0_171-b10 on Linux 4.14.33-51.37.amzn1.x86_64 - Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz - Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 27145 / 27158 0.4 2588.7 1.0X - SQL Json 12969 / 13337 0.8 1236.8 2.1X - SQL Parquet Vectorized 2419 / 2448 4.3 230.7 11.2X - SQL Parquet MR 4631 / 4633 2.3 441.7 5.9X - SQL ORC Vectorized 2412 / 2465 4.3 230.0 11.3X - SQL ORC Vectorized with copy 2633 / 2675 4.0 251.1 10.3X - SQL ORC MR 4280 / 4350 2.4 408.2 6.3X - */ benchmark.run() } } } def repeatedStringScanBenchmark(values: Int): Unit = { - val benchmark = new Benchmark("Repeated String", values) + val benchmark = new Benchmark("Repeated String", values, output = output) withTempPath { dir => withTempTable("t1", "csvTable", "jsonTable", "parquetTable", "orcTable") { @@ -432,26 +324,13 @@ object DataSourceReadBenchmark extends SQLHelper { } } - /* - OpenJDK 64-Bit Server VM 1.8.0_171-b10 on Linux 4.14.33-51.37.amzn1.x86_64 - Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz - Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 17345 / 17424 0.6 1654.1 1.0X - SQL Json 8639 / 8664 1.2 823.9 2.0X - SQL Parquet Vectorized 839 / 854 12.5 80.0 20.7X - SQL Parquet MR 1771 / 1775 5.9 168.9 9.8X - SQL ORC Vectorized 550 / 569 19.1 52.4 31.6X - SQL ORC Vectorized with copy 785 / 849 13.4 74.9 22.1X - SQL ORC MR 2168 / 2202 4.8 206.7 8.0X - */ benchmark.run() } } } def partitionTableScanBenchmark(values: Int): Unit = { - val benchmark = new Benchmark("Partitioned Table", values) + val benchmark = new Benchmark("Partitioned Table", values, output = output) withTempPath { dir => withTempTable("t1", "csvTable", "jsonTable", "parquetTable", "orcTable") { @@ -562,40 +441,13 @@ object DataSourceReadBenchmark extends SQLHelper { } } - /* - OpenJDK 64-Bit Server VM 1.8.0_171-b10 on Linux 4.14.33-51.37.amzn1.x86_64 - Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz - Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - Data column - CSV 32613 / 32841 0.5 2073.4 1.0X - Data column - Json 13343 / 13469 1.2 848.3 2.4X - Data column - Parquet Vectorized 302 / 318 52.1 19.2 108.0X - Data column - Parquet MR 2908 / 2924 5.4 184.9 11.2X - Data column - ORC Vectorized 412 / 425 38.1 26.2 79.1X - Data column - ORC Vectorized with copy 442 / 446 35.6 28.1 73.8X - Data column - ORC MR 2390 / 2396 6.6 152.0 13.6X - Partition column - CSV 9626 / 9683 1.6 612.0 3.4X - Partition column - Json 10909 / 10923 1.4 693.6 3.0X - Partition column - Parquet Vectorized 69 / 76 228.4 4.4 473.6X - Partition column - Parquet MR 1898 / 1933 8.3 120.7 17.2X - Partition column - ORC Vectorized 67 / 74 236.0 4.2 489.4X - Partition column - ORC Vectorized with copy 65 / 72 241.9 4.1 501.6X - Partition column - ORC MR 1743 / 1749 9.0 110.8 18.7X - Both columns - CSV 35523 / 35552 0.4 2258.5 0.9X - Both columns - Json 13676 / 13681 1.2 869.5 2.4X - Both columns - Parquet Vectorized 317 / 326 49.5 20.2 102.7X - Both columns - Parquet MR 3333 / 3336 4.7 211.9 9.8X - Both columns - ORC Vectorized 441 / 446 35.6 28.1 73.9X - Both column - ORC Vectorized with copy 517 / 524 30.4 32.9 63.1X - Both columns - ORC MR 2574 / 2577 6.1 163.6 12.7X - */ benchmark.run() } } } def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = { - val benchmark = new Benchmark("String with Nulls Scan", values) + val benchmark = new Benchmark("String with Nulls Scan", values, output = output) withTempPath { dir => withTempTable("t1", "csvTable", "jsonTable", "parquetTable", "orcTable") { @@ -673,51 +525,16 @@ object DataSourceReadBenchmark extends SQLHelper { } } - /* - OpenJDK 64-Bit Server VM 1.8.0_171-b10 on Linux 4.14.33-51.37.amzn1.x86_64 - Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz - String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 14875 / 14920 0.7 1418.6 1.0X - SQL Json 10974 / 10992 1.0 1046.5 1.4X - SQL Parquet Vectorized 1711 / 1750 6.1 163.2 8.7X - SQL Parquet MR 3838 / 3884 2.7 366.0 3.9X - ParquetReader Vectorized 1155 / 1168 9.1 110.2 12.9X - SQL ORC Vectorized 1341 / 1380 7.8 127.9 11.1X - SQL ORC Vectorized with copy 1659 / 1716 6.3 158.2 9.0X - SQL ORC MR 3594 / 3634 2.9 342.7 4.1X - - - String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 17219 / 17264 0.6 1642.1 1.0X - SQL Json 8843 / 8864 1.2 843.3 1.9X - SQL Parquet Vectorized 1169 / 1178 9.0 111.4 14.7X - SQL Parquet MR 2676 / 2697 3.9 255.2 6.4X - ParquetReader Vectorized 1068 / 1071 9.8 101.8 16.1X - SQL ORC Vectorized 1319 / 1319 7.9 125.8 13.1X - SQL ORC Vectorized with copy 1638 / 1639 6.4 156.2 10.5X - SQL ORC MR 3230 / 3257 3.2 308.1 5.3X - - - String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 13976 / 14053 0.8 1332.8 1.0X - SQL Json 5166 / 5176 2.0 492.6 2.7X - SQL Parquet Vectorized 274 / 282 38.2 26.2 50.9X - SQL Parquet MR 1553 / 1555 6.8 148.1 9.0X - ParquetReader Vectorized 241 / 246 43.5 23.0 57.9X - SQL ORC Vectorized 476 / 479 22.0 45.4 29.3X - SQL ORC Vectorized with copy 584 / 588 17.9 55.7 23.9X - SQL ORC MR 1720 / 1734 6.1 164.1 8.1X - */ benchmark.run() } } } def columnsBenchmark(values: Int, width: Int): Unit = { - val benchmark = new Benchmark(s"Single Column Scan from $width columns", values) + val benchmark = new Benchmark( + s"Single Column Scan from $width columns", + values, + output = output) withTempPath { dir => withTempTable("t1", "csvTable", "jsonTable", "parquetTable", "orcTable") { @@ -763,58 +580,35 @@ object DataSourceReadBenchmark extends SQLHelper { } } - /* - OpenJDK 64-Bit Server VM 1.8.0_171-b10 on Linux 4.14.33-51.37.amzn1.x86_64 - Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz - Single Column Scan from 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 3478 / 3481 0.3 3316.4 1.0X - SQL Json 2646 / 2654 0.4 2523.6 1.3X - SQL Parquet Vectorized 67 / 72 15.8 63.5 52.2X - SQL Parquet MR 207 / 214 5.1 197.6 16.8X - SQL ORC Vectorized 69 / 76 15.2 66.0 50.3X - SQL ORC Vectorized with copy 70 / 76 15.0 66.5 49.9X - SQL ORC MR 299 / 303 3.5 285.1 11.6X - - - Single Column Scan from 50 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 9214 / 9236 0.1 8786.7 1.0X - SQL Json 9943 / 9978 0.1 9482.7 0.9X - SQL Parquet Vectorized 77 / 86 13.6 73.3 119.8X - SQL Parquet MR 229 / 235 4.6 218.6 40.2X - SQL ORC Vectorized 84 / 96 12.5 80.0 109.9X - SQL ORC Vectorized with copy 83 / 91 12.6 79.4 110.7X - SQL ORC MR 843 / 854 1.2 804.0 10.9X - - - Single Column Scan from 100 columns Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - SQL CSV 16503 / 16622 0.1 15738.9 1.0X - SQL Json 19109 / 19184 0.1 18224.2 0.9X - SQL Parquet Vectorized 99 / 108 10.6 94.3 166.8X - SQL Parquet MR 253 / 264 4.1 241.6 65.1X - SQL ORC Vectorized 107 / 114 9.8 101.6 154.8X - SQL ORC Vectorized with copy 107 / 118 9.8 102.1 154.1X - SQL ORC MR 1526 / 1529 0.7 1455.3 10.8X - */ benchmark.run() } } } - def main(args: Array[String]): Unit = { - Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType).foreach { dataType => - numericScanBenchmark(1024 * 1024 * 15, dataType) + override def runBenchmarkSuite(): Unit = { + runBenchmark("SQL Single Numeric Column Scan") { + Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType).foreach { + dataType => numericScanBenchmark(1024 * 1024 * 15, dataType) + } + } + runBenchmark("Int and String Scan") { + intStringScanBenchmark(1024 * 1024 * 10) } - intStringScanBenchmark(1024 * 1024 * 10) - repeatedStringScanBenchmark(1024 * 1024 * 10) - partitionTableScanBenchmark(1024 * 1024 * 15) - for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { - stringWithNullsScanBenchmark(1024 * 1024 * 10, fractionOfNulls) + runBenchmark("Repeated String Scan") { + repeatedStringScanBenchmark(1024 * 1024 * 10) } - for (columnWidth <- List(10, 50, 100)) { - columnsBenchmark(1024 * 1024 * 1, columnWidth) + runBenchmark("Partitioned Table Scan") { + partitionTableScanBenchmark(1024 * 1024 * 15) + } + runBenchmark("String with Nulls Scan") { + for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { + stringWithNullsScanBenchmark(1024 * 1024 * 10, fractionOfNulls) + } + } + runBenchmark("Single Column Scan From Wide Columns") { + for (columnWidth <- List(10, 50, 100)) { + columnsBenchmark(1024 * 1024 * 1, columnWidth) + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala index f44da242e62b..43380869fefe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala @@ -21,247 +21,140 @@ import org.apache.spark.benchmark.Benchmark /** * Benchmark to measure whole stage codegen performance. - * To run this: - * build/sbt "sql/test-only *benchmark.MiscBenchmark" - * - * Benchmarks in this file are skipped in normal builds. + * 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/MiscBenchmark-results.txt". + * }}} */ -class MiscBenchmark extends BenchmarkWithCodegen { +object MiscBenchmark extends SqlBasedBenchmark { - ignore("filter & aggregate without group") { - val N = 500L << 22 - runBenchmark("range/filter/sum", N) { - sparkSession.range(N).filter("(id & 1) = 1").groupBy().sum().collect() + def filterAndAggregateWithoutGroup(numRows: Long): Unit = { + runBenchmark("filter & aggregate without group") { + codegenBenchmark("range/filter/sum", numRows) { + spark.range(numRows).filter("(id & 1) = 1").groupBy().sum().collect() + } } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - range/filter/sum codegen=false 30663 / 31216 68.4 14.6 1.0X - range/filter/sum codegen=true 2399 / 2409 874.1 1.1 12.8X - */ } - ignore("range/limit/sum") { - val N = 500L << 20 - runBenchmark("range/limit/sum", N) { - sparkSession.range(N).limit(1000000).groupBy().sum().collect() + def limitAndAggregateWithoutGroup(numRows: Long): Unit = { + runBenchmark("range/limit/sum") { + codegenBenchmark("range/limit/sum", numRows) { + spark.range(numRows).limit(1000000).groupBy().sum().collect() + } } - /* - Westmere E56xx/L56xx/X56xx (Nehalem-C) - range/limit/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - range/limit/sum codegen=false 609 / 672 861.6 1.2 1.0X - range/limit/sum codegen=true 561 / 621 935.3 1.1 1.1X - */ } - ignore("sample") { - val N = 500 << 18 - runBenchmark("sample with replacement", N) { - sparkSession.range(N).sample(withReplacement = true, 0.01).groupBy().sum().collect() - } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + def sample(numRows: Int): Unit = { + runBenchmark("sample") { + codegenBenchmark("sample with replacement", numRows) { + spark.range(numRows).sample(withReplacement = true, 0.01).groupBy().sum().collect() + } - sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - sample with replacement codegen=false 7073 / 7227 18.5 54.0 1.0X - sample with replacement codegen=true 5199 / 5203 25.2 39.7 1.4X - */ - - runBenchmark("sample without replacement", N) { - sparkSession.range(N).sample(withReplacement = false, 0.01).groupBy().sum().collect() + codegenBenchmark("sample without replacement", numRows) { + spark.range(numRows).sample(withReplacement = false, 0.01).groupBy().sum().collect() + } } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - sample without replacement codegen=false 1508 / 1529 86.9 11.5 1.0X - sample without replacement codegen=true 644 / 662 203.5 4.9 2.3X - */ } - ignore("collect") { - val N = 1 << 20 - - val benchmark = new Benchmark("collect", N) - benchmark.addCase("collect 1 million") { iter => - sparkSession.range(N).collect() - } - benchmark.addCase("collect 2 millions") { iter => - sparkSession.range(N * 2).collect() - } - benchmark.addCase("collect 4 millions") { iter => - sparkSession.range(N * 4).collect() + def collect(numRows: Int): Unit = { + runBenchmark("collect") { + val benchmark = new Benchmark("collect", numRows, output = output) + benchmark.addCase("collect 1 million") { iter => + spark.range(numRows).collect() + } + benchmark.addCase("collect 2 millions") { iter => + spark.range(numRows * 2).collect() + } + benchmark.addCase("collect 4 millions") { iter => + spark.range(numRows * 4).collect() + } + benchmark.run() } - benchmark.run() - - /* - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - collect 1 million 439 / 654 2.4 418.7 1.0X - collect 2 millions 961 / 1907 1.1 916.4 0.5X - collect 4 millions 3193 / 3895 0.3 3044.7 0.1X - */ } - ignore("collect limit") { - val N = 1 << 20 - - val benchmark = new Benchmark("collect limit", N) - benchmark.addCase("collect limit 1 million") { iter => - sparkSession.range(N * 4).limit(N).collect() + def collectLimit(numRows: Int): Unit = { + runBenchmark("collect limit") { + val benchmark = new Benchmark("collect limit", numRows, output = output) + benchmark.addCase("collect limit 1 million") { iter => + spark.range(numRows * 4).limit(numRows).collect() + } + benchmark.addCase("collect limit 2 millions") { iter => + spark.range(numRows * 4).limit(numRows * 2).collect() + } + benchmark.run() } - benchmark.addCase("collect limit 2 millions") { iter => - sparkSession.range(N * 4).limit(N * 2).collect() - } - benchmark.run() - - /* - model name : Westmere E56xx/L56xx/X56xx (Nehalem-C) - collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - collect limit 1 million 833 / 1284 1.3 794.4 1.0X - collect limit 2 millions 3348 / 4005 0.3 3193.3 0.2X - */ } - ignore("generate explode") { - val N = 1 << 24 - runBenchmark("generate explode array", N) { - val df = sparkSession.range(N).selectExpr( - "id as key", - "array(rand(), rand(), rand(), rand(), rand()) as values") - df.selectExpr("key", "explode(values) value").count() - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 - Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz - - generate explode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - generate explode array wholestage off 6920 / 7129 2.4 412.5 1.0X - generate explode array wholestage on 623 / 646 26.9 37.1 11.1X - */ - - runBenchmark("generate explode map", N) { - val df = sparkSession.range(N).selectExpr( - "id as key", - "map('a', rand(), 'b', rand(), 'c', rand(), 'd', rand(), 'e', rand()) pairs") - df.selectExpr("key", "explode(pairs) as (k, v)").count() - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 - Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz - - generate explode map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - generate explode map wholestage off 11978 / 11993 1.4 714.0 1.0X - generate explode map wholestage on 866 / 919 19.4 51.6 13.8X - */ - - runBenchmark("generate posexplode array", N) { - val df = sparkSession.range(N).selectExpr( - "id as key", - "array(rand(), rand(), rand(), rand(), rand()) as values") - df.selectExpr("key", "posexplode(values) as (idx, value)").count() + def explode(numRows: Int): Unit = { + runBenchmark("generate explode") { + codegenBenchmark("generate explode array", numRows) { + val df = spark.range(numRows).selectExpr( + "id as key", + "array(rand(), rand(), rand(), rand(), rand()) as values") + df.selectExpr("key", "explode(values) value").count() + } + + codegenBenchmark("generate explode map", numRows) { + val df = spark.range(numRows).selectExpr( + "id as key", + "map('a', rand(), 'b', rand(), 'c', rand(), 'd', rand(), 'e', rand()) pairs") + df.selectExpr("key", "explode(pairs) as (k, v)").count() + } + + codegenBenchmark("generate posexplode array", numRows) { + val df = spark.range(numRows).selectExpr( + "id as key", + "array(rand(), rand(), rand(), rand(), rand()) as values") + df.selectExpr("key", "posexplode(values) as (idx, value)").count() + } + + codegenBenchmark("generate inline array", numRows) { + val df = spark.range(numRows).selectExpr( + "id as key", + "array((rand(), rand()), (rand(), rand()), (rand(), 0.0d)) as values") + df.selectExpr("key", "inline(values) as (r1, r2)").count() + } + + val M = 60000 + codegenBenchmark("generate big struct array", M) { + import spark.implicits._ + val df = spark.sparkContext.parallelize(Seq(("1", + Array.fill(M)({ + val i = math.random + (i.toString, (i + 1).toString, (i + 2).toString, (i + 3).toString) + })))).toDF("col", "arr") + + df.selectExpr("*", "explode(arr) as arr_col") + .select("col", "arr_col.*").count + } } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 - Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz - - generate posexplode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - generate posexplode array wholestage off 7502 / 7513 2.2 447.1 1.0X - generate posexplode array wholestage on 617 / 623 27.2 36.8 12.2X - */ - - runBenchmark("generate inline array", N) { - val df = sparkSession.range(N).selectExpr( - "id as key", - "array((rand(), rand()), (rand(), rand()), (rand(), 0.0d)) as values") - df.selectExpr("key", "inline(values) as (r1, r2)").count() - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 - Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz - - generate inline array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - generate inline array wholestage off 6901 / 6928 2.4 411.3 1.0X - generate inline array wholestage on 1001 / 1010 16.8 59.7 6.9X - */ - - val M = 60000 - runBenchmark("generate big struct array", M) { - import sparkSession.implicits._ - val df = sparkSession.sparkContext.parallelize(Seq(("1", - Array.fill(M)({ - val i = math.random - (i.toString, (i + 1).toString, (i + 2).toString, (i + 3).toString) - })))).toDF("col", "arr") - - df.selectExpr("*", "expode(arr) as arr_col") - .select("col", "arr_col.*").count - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 - Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz - - test the impact of adding the optimization of Generate.unrequiredChildIndex, - we can see enormous improvement of x250 in this case! and it grows O(n^2). - - with Optimization ON: - - generate big struct array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - generate big struct array wholestage off 331 / 378 0.2 5524.9 1.0X - generate big struct array wholestage on 205 / 232 0.3 3413.1 1.6X - - with Optimization OFF: - - generate big struct array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - generate big struct array wholestage off 49697 / 51496 0.0 828277.7 1.0X - generate big struct array wholestage on 50558 / 51434 0.0 842641.6 1.0X - */ - } - ignore("generate regular generator") { - val N = 1 << 24 - runBenchmark("generate stack", N) { - val df = sparkSession.range(N).selectExpr( - "id as key", - "id % 2 as t1", - "id % 3 as t2", - "id % 5 as t3", - "id % 7 as t4", - "id % 13 as t5") - df.selectExpr("key", "stack(4, t1, t2, t3, t4, t5)").count() + def stack(numRows: Int): Unit = { + runBenchmark("generate regular generator") { + codegenBenchmark("generate stack", numRows) { + val df = spark.range(numRows).selectExpr( + "id as key", + "id % 2 as t1", + "id % 3 as t2", + "id % 5 as t3", + "id % 7 as t4", + "id % 13 as t5") + df.selectExpr("key", "stack(4, t1, t2, t3, t4, t5)").count() + } } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 - Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz - - generate stack: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - generate stack wholestage off 12953 / 13070 1.3 772.1 1.0X - generate stack wholestage on 836 / 847 20.1 49.8 15.5X - */ } + override def runBenchmarkSuite(): Unit = { + filterAndAggregateWithoutGroup(500L << 22) + limitAndAggregateWithoutGroup(500L << 20) + sample(500 << 18) + collect(1 << 20) + collectLimit(1 << 20) + explode(1 << 24) + stack(1 << 24) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala index 18bb4bfe661c..49e7af4a9896 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -23,7 +23,7 @@ import java.net.URI import scala.collection.mutable import scala.language.reflectiveCalls -import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} +import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path, RawLocalFileSystem} import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.catalyst.util._ @@ -248,6 +248,26 @@ class FileIndexSuite extends SharedSQLContext { assert(spark.read.parquet(path.getAbsolutePath).schema.exists(_.name == colToUnescape)) } } + + test("SPARK-25062 - InMemoryFileIndex stores BlockLocation objects no matter what subclass " + + "the FS returns") { + withSQLConf("fs.file.impl" -> classOf[SpecialBlockLocationFileSystem].getName) { + withTempDir { dir => + val file = new File(dir, "text.txt") + stringToFile(file, "text") + + val inMemoryFileIndex = new InMemoryFileIndex( + spark, Seq(new Path(file.getCanonicalPath)), Map.empty, None) { + def leafFileStatuses = leafFiles.values + } + val blockLocations = inMemoryFileIndex.leafFileStatuses.flatMap( + _.asInstanceOf[LocatedFileStatus].getBlockLocations) + + assert(blockLocations.forall(_.getClass == classOf[BlockLocation])) + } + } + } + } class FakeParentPathFileSystem extends RawLocalFileSystem { @@ -257,3 +277,20 @@ class FakeParentPathFileSystem extends RawLocalFileSystem { URI.create("mockFs://some-bucket") } } + +class SpecialBlockLocationFileSystem extends RawLocalFileSystem { + + class SpecialBlockLocation( + names: Array[String], + hosts: Array[String], + offset: Long, + length: Long) + extends BlockLocation(names, hosts, offset, length) + + override def getFileBlockLocations( + file: FileStatus, + start: Long, + len: Long): Array[BlockLocation] = { + Array(new SpecialBlockLocation(Array("dummy"), Array("dummy"), 0L, file.getLen)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index f70df0bcecde..5d4746cf90b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -1820,4 +1820,10 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te checkAnswer(spark.read.option("multiLine", true).schema(schema).csv(input), Row(null)) assert(spark.read.csv(input).collect().toSet == Set(Row())) } + + test("field names of inferred schema shouldn't compare to the first row") { + val input = Seq("1,2").toDS() + val df = spark.read.option("enforceSchema", false).csv(input) + checkAnswer(df, Row("1", "2")) + } } 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 8680b86517b1..ee12f3089243 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 @@ -358,7 +358,7 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { } } - test("SPARK-12218 Converting conjunctions into ORC SearchArguments") { + test("SPARK-12218 and SPARK-25699 Converting conjunctions into ORC SearchArguments") { import org.apache.spark.sql.sources._ // The `LessThan` should be converted while the `StringContains` shouldn't val schema = new StructType( @@ -382,5 +382,40 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { )) )).get.toString } + + // Can not remove unsupported `StringContains` predicate since it is under `Or` operator. + assert(OrcFilters.createFilter(schema, Array( + Or( + LessThan("a", 10), + And( + StringContains("b", "prefix"), + GreaterThan("a", 1) + ) + ) + )).isEmpty) + + // Safely remove unsupported `StringContains` predicate and push down `LessThan` + assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { + OrcFilters.createFilter(schema, Array( + And( + LessThan("a", 10), + StringContains("b", "prefix") + ) + )).get.toString + } + + // Safely remove unsupported `StringContains` predicate, push down `LessThan` and `GreaterThan`. + assertResult("leaf-0 = (LESS_THAN a 10), leaf-1 = (LESS_THAN_EQUALS a 1)," + + " expr = (and leaf-0 (not leaf-1))") { + OrcFilters.createFilter(schema, Array( + And( + And( + LessThan("a", 10), + StringContains("b", "prefix") + ), + GreaterThan("a", 1) + ) + )).get.toString + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index b6bb1d7ba4ce..dc81c0585bf1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.orc.OrcConf.COMPRESS import org.apache.orc.OrcFile +import org.apache.orc.OrcProto.ColumnEncoding.Kind.{DICTIONARY_V2, DIRECT, DIRECT_V2} import org.apache.orc.OrcProto.Stream.Kind import org.apache.orc.impl.RecordReaderImpl import org.scalatest.BeforeAndAfterAll @@ -115,6 +116,76 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } + protected def testSelectiveDictionaryEncoding(isSelective: Boolean) { + val tableName = "orcTable" + + withTempDir { dir => + withTable(tableName) { + val sqlStatement = orcImp match { + case "native" => + s""" + |CREATE TABLE $tableName (zipcode STRING, uniqColumn STRING, value DOUBLE) + |USING ORC + |OPTIONS ( + | path '${dir.toURI}', + | orc.dictionary.key.threshold '1.0', + | orc.column.encoding.direct 'uniqColumn' + |) + """.stripMargin + case "hive" => + s""" + |CREATE TABLE $tableName (zipcode STRING, uniqColumn STRING, value DOUBLE) + |STORED AS ORC + |LOCATION '${dir.toURI}' + |TBLPROPERTIES ( + | orc.dictionary.key.threshold '1.0', + | hive.exec.orc.dictionary.key.size.threshold '1.0', + | orc.column.encoding.direct 'uniqColumn' + |) + """.stripMargin + case impl => + throw new UnsupportedOperationException(s"Unknown ORC implementation: $impl") + } + + sql(sqlStatement) + sql(s"INSERT INTO $tableName VALUES ('94086', 'random-uuid-string', 0.0)") + + val partFiles = dir.listFiles() + .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) + assert(partFiles.length === 1) + + val orcFilePath = new Path(partFiles.head.getAbsolutePath) + val readerOptions = OrcFile.readerOptions(new Configuration()) + val reader = OrcFile.createReader(orcFilePath, readerOptions) + var recordReader: RecordReaderImpl = null + try { + recordReader = reader.rows.asInstanceOf[RecordReaderImpl] + + // Check the kind + val stripe = recordReader.readStripeFooter(reader.getStripes.get(0)) + + // The encodings are divided into direct or dictionary-based categories and + // further refined as to whether they use RLE v1 or v2. RLE v1 is used by + // Hive 0.11 and RLE v2 is introduced in Hive 0.12 ORC with more improvements. + // For more details, see https://orc.apache.org/specification/ + assert(stripe.getColumns(1).getKind === DICTIONARY_V2) + if (isSelective) { + assert(stripe.getColumns(2).getKind === DIRECT_V2) + } else { + assert(stripe.getColumns(2).getKind === DICTIONARY_V2) + } + // Floating point types are stored with DIRECT encoding in IEEE 754 floating + // point bit layout. + assert(stripe.getColumns(3).getKind === DIRECT) + } finally { + if (recordReader != null) { + recordReader.close() + } + } + } + } + } + test("create temporary orc table") { checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_source"), Row(10)) @@ -284,4 +355,8 @@ class OrcSourceSuite extends OrcSuite with SharedSQLContext { test("Check BloomFilter creation") { testBloomFilterCreation(Kind.BLOOM_FILTER_UTF8) // After ORC-101 } + + test("Enforce direct encoding column-wise selectively") { + testSelectiveDictionaryEncoding(isSelective = true) + } } 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 085a44548848..81db3e137964 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 @@ -19,12 +19,15 @@ package org.apache.spark.sql.execution.metric import java.io.File +import scala.reflect.{classTag, ClassTag} import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -518,56 +521,80 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared testMetricsDynamicPartition("parquet", "parquet", "t1") } + private def collectNodeWithinWholeStage[T <: SparkPlan : ClassTag](plan: SparkPlan): Seq[T] = { + val stages = plan.collect { + case w: WholeStageCodegenExec => w + } + assert(stages.length == 1, "The query plan should have one and only one whole-stage.") + + val cls = classTag[T].runtimeClass + stages.head.collect { + case n if n.getClass == cls => n.asInstanceOf[T] + } + } + test("SPARK-25602: SparkPlan.getByteArrayRdd should not consume the input when not necessary") { def checkFilterAndRangeMetrics( df: DataFrame, filterNumOutputs: Int, rangeNumOutputs: Int): Unit = { - var filter: FilterExec = null - var range: RangeExec = null - val collectFilterAndRange: SparkPlan => Unit = { - case f: FilterExec => - assert(filter == null, "the query should only have one Filter") - filter = f - case r: RangeExec => - assert(range == null, "the query should only have one Range") - range = r - case _ => - } - if (SQLConf.get.wholeStageEnabled) { - df.queryExecution.executedPlan.foreach { - case w: WholeStageCodegenExec => - w.child.foreach(collectFilterAndRange) - case _ => - } - } else { - df.queryExecution.executedPlan.foreach(collectFilterAndRange) - } + val plan = df.queryExecution.executedPlan - assert(filter != null && range != null, "the query doesn't have Filter and Range") - assert(filter.metrics("numOutputRows").value == filterNumOutputs) - assert(range.metrics("numOutputRows").value == rangeNumOutputs) + val filters = collectNodeWithinWholeStage[FilterExec](plan) + assert(filters.length == 1, "The query plan should have one and only one Filter") + assert(filters.head.metrics("numOutputRows").value == filterNumOutputs) + + val ranges = collectNodeWithinWholeStage[RangeExec](plan) + assert(ranges.length == 1, "The query plan should have one and only one Range") + assert(ranges.head.metrics("numOutputRows").value == rangeNumOutputs) } - val df = spark.range(0, 3000, 1, 2).toDF().filter('id % 3 === 0) - val df2 = df.limit(2) - Seq(true, false).foreach { wholeStageEnabled => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> wholeStageEnabled.toString) { - df.collect() - checkFilterAndRangeMetrics(df, filterNumOutputs = 1000, rangeNumOutputs = 3000) - - df.queryExecution.executedPlan.foreach(_.resetMetrics()) - // For each partition, we get 2 rows. Then the Filter should produce 2 rows per-partition, - // and Range should produce 1000 rows (one batch) per-partition. Totally Filter produces - // 4 rows, and Range produces 2000 rows. - df.queryExecution.toRdd.mapPartitions(_.take(2)).collect() - checkFilterAndRangeMetrics(df, filterNumOutputs = 4, rangeNumOutputs = 2000) - - // Top-most limit will call `CollectLimitExec.executeCollect`, which will only run the first - // task, so totally the Filter produces 2 rows, and Range produces 1000 rows (one batch). - df2.collect() - checkFilterAndRangeMetrics(df2, filterNumOutputs = 2, rangeNumOutputs = 1000) - } + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + val df = spark.range(0, 3000, 1, 2).toDF().filter('id % 3 === 0) + df.collect() + checkFilterAndRangeMetrics(df, filterNumOutputs = 1000, rangeNumOutputs = 3000) + + df.queryExecution.executedPlan.foreach(_.resetMetrics()) + // For each partition, we get 2 rows. Then the Filter should produce 2 rows per-partition, + // and Range should produce 1000 rows (one batch) per-partition. Totally Filter produces + // 4 rows, and Range produces 2000 rows. + df.queryExecution.toRdd.mapPartitions(_.take(2)).collect() + checkFilterAndRangeMetrics(df, filterNumOutputs = 4, rangeNumOutputs = 2000) + + // Top-most limit will call `CollectLimitExec.executeCollect`, which will only run the first + // task, so totally the Filter produces 2 rows, and Range produces 1000 rows (one batch). + val df2 = df.limit(2) + df2.collect() + checkFilterAndRangeMetrics(df2, filterNumOutputs = 2, rangeNumOutputs = 1000) + } + } + + test("SPARK-25497: LIMIT within whole stage codegen should not consume all the inputs") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + // A special query that only has one partition, so there is no shuffle and the entire query + // can be whole-stage-codegened. + val df = spark.range(0, 1500, 1, 1).limit(10).groupBy('id).count().limit(1).filter('id >= 0) + df.collect() + val plan = df.queryExecution.executedPlan + + val ranges = collectNodeWithinWholeStage[RangeExec](plan) + assert(ranges.length == 1, "The query plan should have one and only one Range") + // The Range should only produce the first batch, i.e. 1000 rows. + assert(ranges.head.metrics("numOutputRows").value == 1000) + + val aggs = collectNodeWithinWholeStage[HashAggregateExec](plan) + assert(aggs.length == 2, "The query plan should have two and only two Aggregate") + val partialAgg = aggs.filter(_.aggregateExpressions.head.mode == Partial).head + // The partial aggregate should output 10 rows, because its input is 10 rows. + assert(partialAgg.metrics("numOutputRows").value == 10) + val finalAgg = aggs.filter(_.aggregateExpressions.head.mode == Final).head + // The final aggregate should only produce 1 row, because the upstream limit only needs 1 row. + assert(finalAgg.metrics("numOutputRows").value == 1) + + val filters = collectNodeWithinWholeStage[FilterExec](plan) + assert(filters.length == 1, "The query plan should have one and only one Filter") + // The final Filter should produce 1 rows, because the input is just one row. + assert(filters.head.metrics("numOutputRows").value == 1) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala index 71dff443e883..3e9ccb0f705d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala @@ -99,11 +99,12 @@ class ForeachBatchSinkSuite extends StreamTest { } assert(ex1.getMessage.contains("foreachBatch function cannot be null")) val ex2 = intercept[AnalysisException] { - ds.writeStream.foreachBatch((_, _) => {}).trigger(Trigger.Continuous("1 second")).start() + ds.writeStream.foreachBatch((_: Dataset[Int], _: Long) => {}) + .trigger(Trigger.Continuous("1 second")).start() } assert(ex2.getMessage.contains("'foreachBatch' is not supported with continuous trigger")) val ex3 = intercept[AnalysisException] { - ds.writeStream.foreachBatch((_, _) => {}).partitionBy("value").start() + ds.writeStream.foreachBatch((_: Dataset[Int], _: Long) => {}).partitionBy("value").start() } assert(ex3.getMessage.contains("'foreachBatch' does not support partitioning")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index a9414200e70f..a2bc651bb2bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.sources import java.io.File import java.net.URI +import scala.util.Random + import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions @@ -47,11 +49,13 @@ class BucketedReadWithoutHiveSupportSuite extends BucketedReadSuite with SharedS abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { import testImplicits._ - private lazy val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") + private val maxI = 5 + private val maxJ = 13 + private lazy val df = (0 until 50).map(i => (i % maxI, i % maxJ, i.toString)).toDF("i", "j", "k") private lazy val nullDF = (for { i <- 0 to 50 s <- Seq(null, "a", "b", "c", "d", "e", "f", null, "g") - } yield (i % 5, s, i % 13)).toDF("i", "j", "k") + } yield (i % maxI, s, i % maxJ)).toDF("i", "j", "k") // number of buckets that doesn't yield empty buckets when bucketing on column j on df/nullDF // empty buckets before filtering might hide bugs in pruning logic @@ -66,23 +70,22 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { .bucketBy(8, "j", "k") .saveAsTable("bucketed_table") - for (i <- 0 until 5) { - val table = spark.table("bucketed_table").filter($"i" === i) - val query = table.queryExecution - val output = query.analyzed.output - val rdd = query.toRdd - - assert(rdd.partitions.length == 8) - - val attrs = table.select("j", "k").queryExecution.analyzed.output - val checkBucketId = rdd.mapPartitionsWithIndex((index, rows) => { - val getBucketId = UnsafeProjection.create( - HashPartitioning(attrs, 8).partitionIdExpression :: Nil, - output) - rows.map(row => getBucketId(row).getInt(0) -> index) - }) - checkBucketId.collect().foreach(r => assert(r._1 == r._2)) - } + val bucketValue = Random.nextInt(maxI) + val table = spark.table("bucketed_table").filter($"i" === bucketValue) + val query = table.queryExecution + val output = query.analyzed.output + val rdd = query.toRdd + + assert(rdd.partitions.length == 8) + + val attrs = table.select("j", "k").queryExecution.analyzed.output + val checkBucketId = rdd.mapPartitionsWithIndex((index, rows) => { + val getBucketId = UnsafeProjection.create( + HashPartitioning(attrs, 8).partitionIdExpression :: Nil, + output) + rows.map(row => getBucketId(row).getInt(0) -> index) + }) + checkBucketId.collect().foreach(r => assert(r._1 == r._2)) } } @@ -145,36 +148,36 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { .bucketBy(numBuckets, "j") .saveAsTable("bucketed_table") - for (j <- 0 until 13) { - // Case 1: EqualTo - checkPrunedAnswers( - bucketSpec, - bucketValues = j :: Nil, - filterCondition = $"j" === j, - df) - - // Case 2: EqualNullSafe - checkPrunedAnswers( - bucketSpec, - bucketValues = j :: Nil, - filterCondition = $"j" <=> j, - df) - - // Case 3: In - checkPrunedAnswers( - bucketSpec, - bucketValues = Seq(j, j + 1, j + 2, j + 3), - filterCondition = $"j".isin(j, j + 1, j + 2, j + 3), - df) - - // Case 4: InSet - val inSetExpr = expressions.InSet($"j".expr, Set(j, j + 1, j + 2, j + 3).map(lit(_).expr)) - checkPrunedAnswers( - bucketSpec, - bucketValues = Seq(j, j + 1, j + 2, j + 3), - filterCondition = Column(inSetExpr), - df) - } + val bucketValue = Random.nextInt(maxJ) + // Case 1: EqualTo + checkPrunedAnswers( + bucketSpec, + bucketValues = bucketValue :: Nil, + filterCondition = $"j" === bucketValue, + df) + + // Case 2: EqualNullSafe + checkPrunedAnswers( + bucketSpec, + bucketValues = bucketValue :: Nil, + filterCondition = $"j" <=> bucketValue, + df) + + // Case 3: In + checkPrunedAnswers( + bucketSpec, + bucketValues = Seq(bucketValue, bucketValue + 1, bucketValue + 2, bucketValue + 3), + filterCondition = $"j".isin(bucketValue, bucketValue + 1, bucketValue + 2, bucketValue + 3), + df) + + // Case 4: InSet + val inSetExpr = expressions.InSet($"j".expr, + Set(bucketValue, bucketValue + 1, bucketValue + 2, bucketValue + 3).map(lit(_).expr)) + checkPrunedAnswers( + bucketSpec, + bucketValues = Seq(bucketValue, bucketValue + 1, bucketValue + 2, bucketValue + 3), + filterCondition = Column(inSetExpr), + df) } } @@ -188,13 +191,12 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { .bucketBy(numBuckets, "j") .saveAsTable("bucketed_table") - for (j <- 0 until 13) { - checkPrunedAnswers( - bucketSpec, - bucketValues = j :: Nil, - filterCondition = $"j" === j, - df) - } + val bucketValue = Random.nextInt(maxJ) + checkPrunedAnswers( + bucketSpec, + bucketValues = bucketValue :: Nil, + filterCondition = $"j" === bucketValue, + df) } } @@ -236,40 +238,39 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { .bucketBy(numBuckets, "j") .saveAsTable("bucketed_table") - for (j <- 0 until 13) { - checkPrunedAnswers( - bucketSpec, - bucketValues = j :: Nil, - filterCondition = $"j" === j && $"k" > $"j", - df) - - checkPrunedAnswers( - bucketSpec, - bucketValues = j :: Nil, - filterCondition = $"j" === j && $"i" > j % 5, - df) - - // check multiple bucket values OR condition - checkPrunedAnswers( - bucketSpec, - bucketValues = Seq(j, j + 1), - filterCondition = $"j" === j || $"j" === (j + 1), - df) - - // check bucket value and none bucket value OR condition - checkPrunedAnswers( - bucketSpec, - bucketValues = Nil, - filterCondition = $"j" === j || $"i" === 0, - df) - - // check AND condition in complex expression - checkPrunedAnswers( - bucketSpec, - bucketValues = Seq(j), - filterCondition = ($"i" === 0 || $"k" > $"j") && $"j" === j, - df) - } + val bucketValue = Random.nextInt(maxJ) + checkPrunedAnswers( + bucketSpec, + bucketValues = bucketValue :: Nil, + filterCondition = $"j" === bucketValue && $"k" > $"j", + df) + + checkPrunedAnswers( + bucketSpec, + bucketValues = bucketValue :: Nil, + filterCondition = $"j" === bucketValue && $"i" > bucketValue % 5, + df) + + // check multiple bucket values OR condition + checkPrunedAnswers( + bucketSpec, + bucketValues = Seq(bucketValue, bucketValue + 1), + filterCondition = $"j" === bucketValue || $"j" === (bucketValue + 1), + df) + + // check bucket value and none bucket value OR condition + checkPrunedAnswers( + bucketSpec, + bucketValues = Nil, + filterCondition = $"j" === bucketValue || $"i" === 0, + df) + + // check AND condition in complex expression + checkPrunedAnswers( + bucketSpec, + bucketValues = Seq(bucketValue), + filterCondition = ($"i" === 0 || $"k" > $"j") && $"j" === bucketValue, + df) } } 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 7cc8abc9f042..e8f291af13ba 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 @@ -351,6 +351,24 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { } } } + + test("SPARK-25700: do not read schema when writing in other modes except append mode") { + withTempPath { file => + 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) + } + intercept[SchemaReadAttemptException] { + df.write.format(cls.getName).option("path", path).mode("append").save() + } + } + } } @@ -640,3 +658,14 @@ object SpecificReaderFactory extends PartitionReaderFactory { } } } + +class SchemaReadAttemptException(m: String) extends RuntimeException(m) + +class SimpleWriteOnlyDataSource extends SimpleWritableDataSource { + override def fullSchema(): 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") + } +} 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 a0f4404f4614..a7dfc2d1deac 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 @@ -43,13 +43,13 @@ class SimpleWritableDataSource extends DataSourceV2 with BatchWriteSupportProvider with SessionConfigSupport { - private val schema = new StructType().add("i", "long").add("j", "long") + protected def fullSchema(): StructType = new StructType().add("i", "long").add("j", "long") override def keyPrefix: String = "simpleWritableDataSource" class ReadSupport(path: String, conf: Configuration) extends SimpleReadSupport { - override def fullSchema(): StructType = schema + override def fullSchema(): StructType = SimpleWritableDataSource.this.fullSchema() override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { val dataPath = new Path(path) @@ -116,7 +116,6 @@ class SimpleWritableDataSource extends DataSourceV2 schema: StructType, mode: SaveMode, options: DataSourceOptions): Optional[BatchWriteSupport] = { - assert(DataType.equalsStructurally(schema.asNullable, this.schema.asNullable)) assert(!SparkContext.getActive.get.conf.getBoolean("spark.speculation", false)) val path = new Path(options.get("path").get()) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 7d5738994757..9443fbb4330a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -71,7 +71,7 @@ class HadoopTableReader( // Hadoop honors "mapreduce.job.maps" as hint, // but will ignore when mapreduce.jobtracker.address is "local". - // https://hadoop.apache.org/docs/r2.6.5/hadoop-mapreduce-client/hadoop-mapreduce-client-core/ + // https://hadoop.apache.org/docs/r2.7.6/hadoop-mapreduce-client/hadoop-mapreduce-client-core/ // mapred-default.xml // // In order keep consistency with Hive, we will let it be 0 in local mode also. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 6a90c44a2633..31899370454b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -53,7 +53,7 @@ private[hive] object IsolatedClientLoader extends Logging { sharesHadoopClasses: Boolean = true): IsolatedClientLoader = synchronized { val resolvedVersion = hiveVersion(hiveMetastoreVersion) // We will first try to share Hadoop classes. If we cannot resolve the Hadoop artifact - // with the given version, we will use Hadoop 2.6 and then will not share Hadoop classes. + // with the given version, we will use Hadoop 2.7 and then will not share Hadoop classes. var _sharesHadoopClasses = sharesHadoopClasses val files = if (resolvedVersions.contains((resolvedVersion, hadoopVersion))) { resolvedVersions((resolvedVersion, hadoopVersion)) @@ -65,13 +65,14 @@ private[hive] object IsolatedClientLoader extends Logging { case e: RuntimeException if e.getMessage.contains("hadoop") => // If the error message contains hadoop, it is probably because the hadoop // version cannot be resolved. - logWarning(s"Failed to resolve Hadoop artifacts for the version $hadoopVersion. " + - s"We will change the hadoop version from $hadoopVersion to 2.6.0 and try again. " + - "Hadoop classes will not be shared between Spark and Hive metastore client. " + + val fallbackVersion = "2.7.3" + logWarning(s"Failed to resolve Hadoop artifacts for the version $hadoopVersion. We " + + s"will change the hadoop version from $hadoopVersion to $fallbackVersion and try " + + "again. Hadoop classes will not be shared between Spark and Hive metastore client. " + "It is recommended to set jars used by Hive metastore client through " + "spark.sql.hive.metastore.jars in the production environment.") _sharesHadoopClasses = false - (downloadVersion(resolvedVersion, "2.6.5", ivyPath), "2.6.5") + (downloadVersion(resolvedVersion, fallbackVersion, ivyPath), fallbackVersion) } resolvedVersions.put((resolvedVersion, actualHadoopVersion), downloadedFiles) resolvedVersions((resolvedVersion, actualHadoopVersion)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala index aee9cb58a031..a82576a233ac 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala @@ -79,6 +79,23 @@ private[orc] object OrcFilters extends Logging { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { + createBuilder(dataTypeMap, expression, builder, canPartialPushDownConjuncts = true) + } + + /** + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input filter predicates. + * @param builder the input SearchArgument.Builder. + * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed + * down safely. Pushing ONLY one side of AND down is safe to + * do at the top level or none of its ancestors is NOT and OR. + * @return the builder so far. + */ + private def createBuilder( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder, + canPartialPushDownConjuncts: Boolean): Option[Builder] = { def isSearchableType(dataType: DataType): Boolean = dataType match { // Only the values in the Spark types below can be recognized by // the `SearchArgumentImpl.BuilderImpl.boxLiteral()` method. @@ -90,32 +107,52 @@ private[orc] object OrcFilters extends Logging { expression match { case And(left, right) => - // At here, it is not safe to just convert one side if we do not understand the - // other side. Here is an example used to explain the reason. + // At here, it is not safe to just convert one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to // convert b in ('1'). If we only convert a = 2, we will end up with a filter // NOT(a = 2), which will generate wrong results. - // Pushing one side of AND down is only safe to do at the top level. - // You can see ParquetRelation's initializeLocalJobFunc method as an example. - for { - _ <- buildSearchArgument(dataTypeMap, left, newBuilder) - _ <- buildSearchArgument(dataTypeMap, right, newBuilder) - lhs <- buildSearchArgument(dataTypeMap, left, builder.startAnd()) - rhs <- buildSearchArgument(dataTypeMap, right, lhs) - } yield rhs.end() + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate + // can be safely removed. + val leftBuilderOption = + createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) + val rightBuilderOption = + createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) + (leftBuilderOption, rightBuilderOption) match { + case (Some(_), Some(_)) => + for { + lhs <- createBuilder(dataTypeMap, left, + builder.startAnd(), canPartialPushDownConjuncts) + rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) + } yield rhs.end() + + case (Some(_), None) if canPartialPushDownConjuncts => + createBuilder(dataTypeMap, left, builder, canPartialPushDownConjuncts) + + case (None, Some(_)) if canPartialPushDownConjuncts => + createBuilder(dataTypeMap, right, builder, canPartialPushDownConjuncts) + + case _ => None + } case Or(left, right) => for { - _ <- buildSearchArgument(dataTypeMap, left, newBuilder) - _ <- buildSearchArgument(dataTypeMap, right, newBuilder) - lhs <- buildSearchArgument(dataTypeMap, left, builder.startOr()) - rhs <- buildSearchArgument(dataTypeMap, right, lhs) + _ <- createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts = false) + _ <- createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts = false) + lhs <- createBuilder(dataTypeMap, left, + builder.startOr(), canPartialPushDownConjuncts = false) + rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts = false) } yield rhs.end() case Not(child) => for { - _ <- buildSearchArgument(dataTypeMap, child, newBuilder) - negate <- buildSearchArgument(dataTypeMap, child, builder.startNot()) + _ <- createBuilder(dataTypeMap, child, newBuilder, canPartialPushDownConjuncts = false) + negate <- createBuilder(dataTypeMap, + child, builder.startNot(), canPartialPushDownConjuncts = false) } yield negate.end() // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala index 1bd7e52c88ec..398f4d2efbbf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala @@ -229,8 +229,8 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo tableCompressionCodecs: List[String]) (assertionCompressionCodec: (Option[String], String, String, Long) => Unit): Unit = { withSQLConf(getConvertMetastoreConfName(format) -> convertMetastore.toString) { - tableCompressionCodecs.foreach { tableCompression => - compressionCodecs.foreach { sessionCompressionCodec => + tableCompressionCodecs.zipAll(compressionCodecs, null, "SNAPPY").foreach { + case (tableCompression, sessionCompressionCodec) => withSQLConf(getSparkCompressionConfName(format) -> sessionCompressionCodec) { // 'tableCompression = null' means no table-level compression val compression = Option(tableCompression) @@ -240,7 +240,6 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo compression, sessionCompressionCodec, realCompressionCodec, tableSize) } } - } } } } @@ -262,7 +261,10 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo } } - def checkForTableWithCompressProp(format: String, compressCodecs: List[String]): Unit = { + def checkForTableWithCompressProp( + format: String, + tableCompressCodecs: List[String], + sessionCompressCodecs: List[String]): Unit = { Seq(true, false).foreach { isPartitioned => Seq(true, false).foreach { convertMetastore => Seq(true, false).foreach { usingCTAS => @@ -271,10 +273,10 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo isPartitioned, convertMetastore, usingCTAS, - compressionCodecs = compressCodecs, - tableCompressionCodecs = compressCodecs) { + compressionCodecs = sessionCompressCodecs, + tableCompressionCodecs = tableCompressCodecs) { case (tableCodec, sessionCodec, realCodec, tableSize) => - val expectCodec = tableCodec.get + val expectCodec = tableCodec.getOrElse(sessionCodec) assert(expectCodec == realCodec) assert(checkTableSize( format, expectCodec, isPartitioned, convertMetastore, usingCTAS, tableSize)) @@ -284,36 +286,22 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo } } - def checkForTableWithoutCompressProp(format: String, compressCodecs: List[String]): Unit = { - Seq(true, false).foreach { isPartitioned => - Seq(true, false).foreach { convertMetastore => - Seq(true, false).foreach { usingCTAS => - checkTableCompressionCodecForCodecs( - format, - isPartitioned, - convertMetastore, - usingCTAS, - compressionCodecs = compressCodecs, - tableCompressionCodecs = List(null)) { - case (tableCodec, sessionCodec, realCodec, tableSize) => - // Always expect session-level take effect - assert(sessionCodec == realCodec) - assert(checkTableSize( - format, sessionCodec, isPartitioned, convertMetastore, usingCTAS, tableSize)) - } - } - } - } - } - test("both table-level and session-level compression are set") { - checkForTableWithCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) - checkForTableWithCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) + checkForTableWithCompressProp("parquet", + tableCompressCodecs = List("UNCOMPRESSED", "SNAPPY", "GZIP"), + sessionCompressCodecs = List("SNAPPY", "GZIP", "SNAPPY")) + checkForTableWithCompressProp("orc", + tableCompressCodecs = List("NONE", "SNAPPY", "ZLIB"), + sessionCompressCodecs = List("SNAPPY", "ZLIB", "SNAPPY")) } test("table-level compression is not set but session-level compressions is set ") { - checkForTableWithoutCompressProp("parquet", List("UNCOMPRESSED", "SNAPPY", "GZIP")) - checkForTableWithoutCompressProp("orc", List("NONE", "SNAPPY", "ZLIB")) + checkForTableWithCompressProp("parquet", + tableCompressCodecs = List.empty, + sessionCompressCodecs = List("UNCOMPRESSED", "SNAPPY", "GZIP")) + checkForTableWithCompressProp("orc", + tableCompressCodecs = List.empty, + sessionCompressCodecs = List("NONE", "SNAPPY", "ZLIB")) } def checkTableWriteWithCompressionCodecs(format: String, compressCodecs: List[String]): Unit = { 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 a676cf6ce692..f839e8979d35 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 @@ -33,11 +33,13 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} import org.apache.spark.sql.types.{DecimalType, StructType} +import org.apache.spark.tags.ExtendedHiveTest import org.apache.spark.util.{ResetSystemProperties, Utils} /** * This suite tests spark-submit with applications using HiveContext. */ +@ExtendedHiveTest class HiveSparkSubmitSuite extends SparkSubmitTestUtils with Matchers @@ -46,8 +48,6 @@ class HiveSparkSubmitSuite override protected val enableAutoThreadAudit = false - // TODO: rewrite these or mark them as slow tests to be run sparingly - override def beforeEach() { super.beforeEach() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala index fa9f753795f6..7a325bf26b4c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala @@ -32,7 +32,7 @@ class HiveClientSuite(version: String) private val tryDirectSqlKey = HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname - private val testPartitionCount = 3 * 24 * 4 + private val testPartitionCount = 3 * 5 * 4 private def init(tryDirectSql: Boolean): HiveClient = { val storageFormat = CatalogStorageFormat( @@ -51,7 +51,7 @@ class HiveClientSuite(version: String) val partitions = for { ds <- 20170101 to 20170103 - h <- 0 to 23 + h <- 0 to 4 chunk <- Seq("aa", "ab", "ba", "bb") } yield CatalogTablePartition(Map( "ds" -> ds.toString, @@ -92,7 +92,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("ds") <=> 20170101, 20170101 to 20170103, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } @@ -100,7 +100,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("ds") === 20170101, 20170101 to 20170101, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } @@ -118,7 +118,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("chunk") === "aa", 20170101 to 20170103, - 0 to 23, + 0 to 4, "aa" :: Nil) } @@ -126,7 +126,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("chunk").cast(IntegerType) === 1, 20170101 to 20170103, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } @@ -134,7 +134,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("chunk").cast(BooleanType) === true, 20170101 to 20170103, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } @@ -142,23 +142,23 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( Literal(20170101) === attr("ds"), 20170101 to 20170101, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } - test("getPartitionsByFilter: ds=20170101 and h=10") { + test("getPartitionsByFilter: ds=20170101 and h=2") { testMetastorePartitionFiltering( - attr("ds") === 20170101 && attr("h") === 10, + attr("ds") === 20170101 && attr("h") === 2, 20170101 to 20170101, - 10 to 10, + 2 to 2, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } - test("getPartitionsByFilter: cast(ds as long)=20170101L and h=10") { + test("getPartitionsByFilter: cast(ds as long)=20170101L and h=2") { testMetastorePartitionFiltering( - attr("ds").cast(LongType) === 20170101L && attr("h") === 10, + attr("ds").cast(LongType) === 20170101L && attr("h") === 2, 20170101 to 20170101, - 10 to 10, + 2 to 2, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } @@ -166,7 +166,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("ds") === 20170101 || attr("ds") === 20170102, 20170101 to 20170102, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } @@ -174,7 +174,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("ds").in(20170102, 20170103), 20170102 to 20170103, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } @@ -182,7 +182,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("ds").cast(LongType).in(20170102L, 20170103L), 20170102 to 20170103, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil) } @@ -190,7 +190,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("ds").in(20170102, 20170103), 20170102 to 20170103, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil, { case expr @ In(v, list) if expr.inSetConvertible => InSet(v, list.map(_.eval(EmptyRow)).toSet) @@ -202,7 +202,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("ds").cast(LongType).in(20170102L, 20170103L), 20170102 to 20170103, - 0 to 23, + 0 to 4, "aa" :: "ab" :: "ba" :: "bb" :: Nil, { case expr @ In(v, list) if expr.inSetConvertible => InSet(v, list.map(_.eval(EmptyRow)).toSet) @@ -213,7 +213,7 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("chunk").in("ab", "ba"), 20170101 to 20170103, - 0 to 23, + 0 to 4, "ab" :: "ba" :: Nil) } @@ -221,34 +221,34 @@ class HiveClientSuite(version: String) testMetastorePartitionFiltering( attr("chunk").in("ab", "ba"), 20170101 to 20170103, - 0 to 23, + 0 to 4, "ab" :: "ba" :: Nil, { case expr @ In(v, list) if expr.inSetConvertible => InSet(v, list.map(_.eval(EmptyRow)).toSet) }) } - test("getPartitionsByFilter: (ds=20170101 and h>=8) or (ds=20170102 and h<8)") { - val day1 = (20170101 to 20170101, 8 to 23, Seq("aa", "ab", "ba", "bb")) - val day2 = (20170102 to 20170102, 0 to 7, Seq("aa", "ab", "ba", "bb")) - testMetastorePartitionFiltering((attr("ds") === 20170101 && attr("h") >= 8) || - (attr("ds") === 20170102 && attr("h") < 8), day1 :: day2 :: Nil) + test("getPartitionsByFilter: (ds=20170101 and h>=2) or (ds=20170102 and h<2)") { + val day1 = (20170101 to 20170101, 2 to 4, Seq("aa", "ab", "ba", "bb")) + val day2 = (20170102 to 20170102, 0 to 1, Seq("aa", "ab", "ba", "bb")) + testMetastorePartitionFiltering((attr("ds") === 20170101 && attr("h") >= 2) || + (attr("ds") === 20170102 && attr("h") < 2), day1 :: day2 :: Nil) } - test("getPartitionsByFilter: (ds=20170101 and h>=8) or (ds=20170102 and h<(7+1))") { - val day1 = (20170101 to 20170101, 8 to 23, Seq("aa", "ab", "ba", "bb")) + test("getPartitionsByFilter: (ds=20170101 and h>=2) or (ds=20170102 and h<(1+1))") { + val day1 = (20170101 to 20170101, 2 to 4, Seq("aa", "ab", "ba", "bb")) // Day 2 should include all hours because we can't build a filter for h<(7+1) - val day2 = (20170102 to 20170102, 0 to 23, Seq("aa", "ab", "ba", "bb")) - testMetastorePartitionFiltering((attr("ds") === 20170101 && attr("h") >= 8) || - (attr("ds") === 20170102 && attr("h") < (Literal(7) + 1)), day1 :: day2 :: Nil) + val day2 = (20170102 to 20170102, 0 to 4, Seq("aa", "ab", "ba", "bb")) + testMetastorePartitionFiltering((attr("ds") === 20170101 && attr("h") >= 2) || + (attr("ds") === 20170102 && attr("h") < (Literal(1) + 1)), day1 :: day2 :: Nil) } test("getPartitionsByFilter: " + - "chunk in ('ab', 'ba') and ((ds=20170101 and h>=8) or (ds=20170102 and h<8))") { - val day1 = (20170101 to 20170101, 8 to 23, Seq("ab", "ba")) - val day2 = (20170102 to 20170102, 0 to 7, Seq("ab", "ba")) + "chunk in ('ab', 'ba') and ((ds=20170101 and h>=2) or (ds=20170102 and h<2))") { + val day1 = (20170101 to 20170101, 2 to 4, Seq("ab", "ba")) + val day2 = (20170102 to 20170102, 0 to 1, Seq("ab", "ba")) testMetastorePartitionFiltering(attr("chunk").in("ab", "ba") && - ((attr("ds") === 20170101 && attr("h") >= 8) || (attr("ds") === 20170102 && attr("h") < 8)), + ((attr("ds") === 20170101 && attr("h") >= 2) || (attr("ds") === 20170102 && attr("h") < 2)), day1 :: day2 :: Nil) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index b9c32e789a41..a5cff35abf37 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -740,10 +740,6 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd sql("select key, count(*) c from src group by key having c").collect() } - test("SPARK-2225: turn HAVING without GROUP BY into a simple filter") { - assert(sql("select key from src having key > 490").collect().size < 100) - } - test("union/except/intersect") { assertResult(Array(Row(1), Row(1))) { sql("select 1 as a union all select 1 as a").collect() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala index 0ef630bbd367..c9309197791b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala @@ -416,7 +416,7 @@ class ObjectHashAggregateSuite actual.zip(expected).foreach { case (lhs: Row, rhs: Row) => assert(lhs.length == rhs.length) lhs.toSeq.zip(rhs.toSeq).foreach { - case (a: Double, b: Double) => checkResult(a, b +- tolerance, DoubleType) + case (a: Double, b: Double) => checkResult(a, b +- tolerance, DoubleType, false) case (a, b) => a == b } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala index 283037caf4a9..5094763b0cd2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala @@ -351,7 +351,7 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { } } - test("SPARK-12218 Converting conjunctions into ORC SearchArguments") { + test("SPARK-12218 and SPARK-25699 Converting conjunctions into ORC SearchArguments") { import org.apache.spark.sql.sources._ // The `LessThan` should be converted while the `StringContains` shouldn't val schema = new StructType( @@ -383,5 +383,48 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { )) )).get.toString } + + // Can not remove unsupported `StringContains` predicate since it is under `Or` operator. + assert(OrcFilters.createFilter(schema, Array( + Or( + LessThan("a", 10), + And( + StringContains("b", "prefix"), + GreaterThan("a", 1) + ) + ) + )).isEmpty) + + // Safely remove unsupported `StringContains` predicate and push down `LessThan` + assertResult( + """leaf-0 = (LESS_THAN a 10) + |expr = leaf-0 + """.stripMargin.trim + ) { + OrcFilters.createFilter(schema, Array( + And( + LessThan("a", 10), + StringContains("b", "prefix") + ) + )).get.toString + } + + // Safely remove unsupported `StringContains` predicate, push down `LessThan` and `GreaterThan`. + assertResult( + """leaf-0 = (LESS_THAN a 10) + |leaf-1 = (LESS_THAN_EQUALS a 1) + |expr = (and leaf-0 (not leaf-1)) + """.stripMargin.trim + ) { + OrcFilters.createFilter(schema, Array( + And( + And( + LessThan("a", 10), + StringContains("b", "prefix") + ), + GreaterThan("a", 1) + ) + )).get.toString + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index c1ae2f6861cb..7fefaf53939b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -182,4 +182,12 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { } } } + + test("Enforce direct encoding column-wise selectively") { + Seq(true, false).foreach { convertMetastore => + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore") { + testSelectiveDictionaryEncoding(isSelective = false) + } + } + } } 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 b9ec940ac492..6bd59fde550d 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 @@ -38,6 +38,10 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes val dataSourceName: String + protected val parquetDataSourceName: String = "parquet" + + private def isParquetDataSource: Boolean = dataSourceName == parquetDataSourceName + protected def supportsDataType(dataType: DataType): Boolean = true val dataSchema = @@ -114,10 +118,21 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes new UDT.MyDenseVectorUDT() ).filter(supportsDataType) - for (dataType <- supportedDataTypes) { - for (parquetDictionaryEncodingEnabled <- Seq(true, false)) { - test(s"test all data types - $dataType with parquet.enable.dictionary = " + - s"$parquetDictionaryEncodingEnabled") { + test(s"test all data types") { + val parquetDictionaryEncodingEnabledConfs = if (isParquetDataSource) { + // Run with/without Parquet dictionary encoding enabled for Parquet data source. + Seq(true, false) + } else { + Seq(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 @@ -754,33 +769,6 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } } - // NOTE: This test suite is not super deterministic. On nodes with only relatively few cores - // (4 or even 1), it's hard to reproduce the data loss issue. But on nodes with for example 8 or - // more cores, the issue can be reproduced steadily. Fortunately our Jenkins builder meets this - // requirement. We probably want to move this test case to spark-integration-tests or spark-perf - // later. - test("SPARK-8406: Avoids name collision while writing files") { - withTempPath { dir => - val path = dir.getCanonicalPath - spark - .range(10000) - .repartition(250) - .write - .mode(SaveMode.Overwrite) - .format(dataSourceName) - .save(path) - - assertResult(10000) { - spark - .read - .format(dataSourceName) - .option("dataSchema", StructType(StructField("id", LongType) :: Nil).json) - .load(path) - .count() - } - } - } - test("SPARK-8887: Explicitly define which data types can be used as dynamic partition columns") { val df = Seq( (1, "v1", Array(1, 2, 3), Map("k1" -> "v1"), Tuple2(1, "4")), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala index 6858bbc44172..6ebc1d145848 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.types._ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { import testImplicits._ - override val dataSourceName: String = "parquet" + override val dataSourceName: String = parquetDataSourceName // Parquet does not play well with NullType. override protected def supportsDataType(dataType: DataType): Boolean = dataType match { @@ -232,4 +232,33 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { } } } + + // NOTE: This test suite is not super deterministic. On nodes with only relatively few cores + // (4 or even 1), it's hard to reproduce the data loss issue. But on nodes with for example 8 or + // more cores, the issue can be reproduced steadily. Fortunately our Jenkins builder meets this + // requirement. We probably want to move this test case to spark-integration-tests or spark-perf + // later. + // Also, this test is slow. As now all the file format data source are using common code + // for creating result files, we can test Parquet only to reduce test time. + test("SPARK-8406: Avoids name collision while writing files") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark + .range(10000) + .repartition(250) + .write + .mode(SaveMode.Overwrite) + .format(dataSourceName) + .save(path) + + assertResult(10000) { + spark + .read + .format(dataSourceName) + .option("dataSchema", StructType(StructField("id", LongType) :: Nil).json) + .load(path) + .count() + } + } + } } 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 027403816f53..122f25b21a0d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -537,7 +537,7 @@ class StreamingContext private[streaming] ( ExecutorAllocationManager.isDynamicAllocationEnabled(conf)) { logWarning("Dynamic Allocation is enabled for this application. " + "Enabling Dynamic allocation for Spark Streaming applications can cause data loss if " + - "Write Ahead Log is not enabled for non-replayable sources like Flume. " + + "Write Ahead Log is not enabled for non-replayable sources. " + "See the programming guide for details on how to enable the Write Ahead Log.") } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index a59f4efccb57..99396865f7d2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -30,7 +30,7 @@ import org.apache.spark.streaming.dstream.DStream /** * A Java-friendly interface to [[org.apache.spark.streaming.dstream.DStream]], the basic * abstraction in Spark Streaming that represents a continuous stream of data. - * DStreams can either be created from live data (such as, data from TCP sockets, Kafka, Flume, + * DStreams can either be created from live data (such as, data from TCP sockets, Kafka, * etc.) or it can be generated by transforming existing DStreams using operations such as `map`, * `window`. For operations applicable to key-value pair DStreams, see * [[org.apache.spark.streaming.api.java.JavaPairDStream]]. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 4a4d2c5d9d8c..35243373daf9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -40,7 +40,7 @@ import org.apache.spark.util.{CallSite, Utils} * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous * sequence of RDDs (of the same type) representing a continuous stream of data (see * org.apache.spark.rdd.RDD in the Spark core documentation for more details on RDDs). - * DStreams can either be created from live data (such as, data from TCP sockets, Kafka, Flume, + * DStreams can either be created from live data (such as, data from TCP sockets, Kafka, * etc.) using a [[org.apache.spark.streaming.StreamingContext]] or it can be generated by * transforming existing DStreams using operations such as `map`, * `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each DStream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 931f015f03b6..6495c9124704 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -56,7 +56,6 @@ abstract class InputDStream[T: ClassTag](_ssc: StreamingContext) /** A human-readable name of this InputDStream */ private[streaming] def name: String = { - // e.g. FlumePollingDStream -> "Flume polling stream" val newName = Utils.getFormattedClassName(this) .replaceAll("InputDStream", "Stream") .split("(?=[A-Z])")